@@ -584,26 +584,63 @@ def has_expected_values(actual):
584584
585585 assert_that (result , has_expected_values )
586586
587- def test_combining_with_sliding_windows_and_fanout_raises_error (self ):
587+ def test_combining_with_sliding_windows_and_fanout (self ):
588588 options = PipelineOptions ()
589589 options .view_as (StandardOptions ).streaming = True
590- with self .assertRaises (ValueError ):
591- with TestPipeline (options = options ) as p :
592- _ = (
593- p
594- | beam .Create ([
595- window .TimestampedValue (0 , Timestamp (seconds = 1666707510 )),
596- window .TimestampedValue (1 , Timestamp (seconds = 1666707511 )),
597- window .TimestampedValue (2 , Timestamp (seconds = 1666707512 )),
598- window .TimestampedValue (3 , Timestamp (seconds = 1666707513 )),
599- window .TimestampedValue (5 , Timestamp (seconds = 1666707515 )),
600- window .TimestampedValue (6 , Timestamp (seconds = 1666707516 )),
601- window .TimestampedValue (7 , Timestamp (seconds = 1666707517 )),
602- window .TimestampedValue (8 , Timestamp (seconds = 1666707518 ))
603- ])
604- | beam .WindowInto (window .SlidingWindows (10 , 5 ))
605- | beam .CombineGlobally (beam .combiners .ToListCombineFn ()).
606- without_defaults ().with_fanout (7 ))
590+ with TestPipeline (options = options ) as p :
591+
592+ def has_expected_values (actual ):
593+ from hamcrest .core import assert_that as hamcrest_assert
594+ from hamcrest .library .collection import only_contains
595+ ordered = sorted (actual )
596+ hamcrest_assert (
597+ ordered ,
598+ only_contains ([0 , 1 , 2 , 3 ], [0 , 1 , 2 , 3 , 5 , 6 , 7 , 8 ], [5 , 6 , 7 , 8 ]))
599+
600+ result = (
601+ p
602+ | beam .Create ([
603+ window .TimestampedValue (0 , Timestamp (seconds = 1666707510 )),
604+ window .TimestampedValue (1 , Timestamp (seconds = 1666707511 )),
605+ window .TimestampedValue (2 , Timestamp (seconds = 1666707512 )),
606+ window .TimestampedValue (3 , Timestamp (seconds = 1666707513 )),
607+ window .TimestampedValue (5 , Timestamp (seconds = 1666707515 )),
608+ window .TimestampedValue (6 , Timestamp (seconds = 1666707516 )),
609+ window .TimestampedValue (7 , Timestamp (seconds = 1666707517 )),
610+ window .TimestampedValue (8 , Timestamp (seconds = 1666707518 ))
611+ ])
612+ | beam .WindowInto (window .SlidingWindows (10 , 5 ))
613+ | beam .CombineGlobally (beam .combiners .ToListCombineFn ()).
614+ without_defaults ().with_fanout (7 ))
615+ assert_that (result , has_expected_values )
616+
617+ def test_combining_with_session_windows_and_fanout (self ):
618+ options = PipelineOptions ()
619+ options .view_as (StandardOptions ).streaming = True
620+ with TestPipeline (options = options ) as p :
621+
622+ def has_expected_values (actual ):
623+ from hamcrest .core import assert_that as hamcrest_assert
624+ from hamcrest .library .collection import only_contains
625+ ordered = sorted (actual )
626+ hamcrest_assert (ordered , only_contains ([0 , 1 , 2 , 3 ], [5 , 6 , 7 , 8 ]))
627+
628+ result = (
629+ p
630+ | beam .Create ([
631+ window .TimestampedValue (0 , Timestamp (seconds = 1666707510 )),
632+ window .TimestampedValue (1 , Timestamp (seconds = 1666707511 )),
633+ window .TimestampedValue (2 , Timestamp (seconds = 1666707512 )),
634+ window .TimestampedValue (3 , Timestamp (seconds = 1666707513 )),
635+ window .TimestampedValue (5 , Timestamp (seconds = 1666707515 )),
636+ window .TimestampedValue (6 , Timestamp (seconds = 1666707516 )),
637+ window .TimestampedValue (7 , Timestamp (seconds = 1666707517 )),
638+ window .TimestampedValue (8 , Timestamp (seconds = 1666707518 ))
639+ ])
640+ | beam .WindowInto (window .Sessions (2 ))
641+ | beam .CombineGlobally (beam .combiners .ToListCombineFn ()).
642+ without_defaults ().with_fanout (7 ))
643+ assert_that (result , has_expected_values )
607644
608645 def test_MeanCombineFn_combine (self ):
609646 with TestPipeline () as p :
0 commit comments