3333from apache_beam .options .pipeline_options import PipelineOptions
3434from apache_beam .testing .util import assert_that
3535from apache_beam .testing .util import equal_to
36+ from apache_beam .transforms .core import DeadLetter
3637from apache_beam .transforms .resources import ResourceHint
3738from apache_beam .transforms .userstate import BagStateSpec
3839from apache_beam .transforms .userstate import ReadModifyWriteStateSpec
@@ -559,9 +560,7 @@ def test_with_exception_handling_then_with_outputs(self):
559560 self .assertEqual (results .main .element_type , int )
560561 self .assertEqual (results .threes .element_type , int )
561562 self .assertEqual (results .fives .element_type , str )
562- self .assertEqual (
563- results .bad .element_type ,
564- typehints .Tuple [int , typehints .Tuple [type , str , typehints .List [str ]]])
563+ self .assertEqual (results .bad .element_type , DeadLetter [int ])
565564
566565 def test_with_outputs_then_with_exception_handling (self ):
567566 """Direction 2: .with_outputs().with_exception_handling()"""
@@ -582,9 +581,7 @@ def test_with_outputs_then_with_exception_handling(self):
582581 self .assertEqual (results .main .element_type , int )
583582 self .assertEqual (results .threes .element_type , int )
584583 self .assertEqual (results .fives .element_type , str )
585- self .assertEqual (
586- results .bad .element_type ,
587- typehints .Tuple [int , typehints .Tuple [type , str , typehints .List [str ]]])
584+ self .assertEqual (results .bad .element_type , DeadLetter [int ])
588585
589586 def test_with_outputs_then_with_exception_handling_custom_dead_letter_tag (
590587 self ):
@@ -603,9 +600,7 @@ def test_with_outputs_then_with_exception_handling_custom_dead_letter_tag(
603600 bad_elements = results .errors | beam .Keys ()
604601 assert_that (bad_elements , equal_to ([2 ]), 'errors' )
605602 self .assertEqual (results .threes .element_type , int )
606- self .assertEqual (
607- results .errors .element_type ,
608- typehints .Tuple [int , typehints .Tuple [type , str , typehints .List [str ]]])
603+ self .assertEqual (results .errors .element_type , DeadLetter [int ])
609604
610605 def test_with_exception_handling_then_with_outputs_custom_dead_letter_tag (
611606 self ):
@@ -624,9 +619,7 @@ def test_with_exception_handling_then_with_outputs_custom_dead_letter_tag(
624619 bad_elements = results .errors | beam .Keys ()
625620 assert_that (bad_elements , equal_to ([2 ]), 'errors' )
626621 self .assertEqual (results .threes .element_type , int )
627- self .assertEqual (
628- results .errors .element_type ,
629- typehints .Tuple [int , typehints .Tuple [type , str , typehints .List [str ]]])
622+ self .assertEqual (results .errors .element_type , DeadLetter [int ])
630623
631624 def test_exception_handling_no_with_outputs_backward_compat (self ):
632625 """Without with_outputs(), behavior is unchanged."""
0 commit comments