def setUp(self): self.window1 = IntervalWindow(0, 10) self.window2 = IntervalWindow(10, 20) self.window3 = IntervalWindow(20, 30) self.windowed_value = WindowedValue( 'a', 57, (self.window1, self.window2, self.window3)) self.restriction = OffsetRange(0, 100) self.watermark_estimator_state = Timestamp(21) self.restriction_provider = TestOffsetRestrictionProvider() self.watermark_estimator = ManualWatermarkEstimator(Timestamp(42)) self.maxDiff = None
def test_set_watermark(self): watermark_estimator = ManualWatermarkEstimator(None) self.assertIsNone(watermark_estimator.current_watermark()) watermark_estimator.observe_timestamp(Timestamp(10)) self.assertIsNone(watermark_estimator.current_watermark()) watermark_estimator.set_watermark(Timestamp(20)) self.assertEqual(watermark_estimator.current_watermark(), Timestamp(20)) watermark_estimator.set_watermark(Timestamp(30)) self.assertEqual(watermark_estimator.current_watermark(), Timestamp(30)) with self.assertRaises(ValueError): watermark_estimator.set_watermark(Timestamp(25))
def test_initialization(self): watermark_estimator = ManualWatermarkEstimator(None) self.assertIsNone(watermark_estimator.get_estimator_state()) self.assertIsNone(watermark_estimator.current_watermark()) watermark_estimator = ManualWatermarkEstimator(Timestamp(10)) self.assertEqual(watermark_estimator.get_estimator_state(), Timestamp(10))
def test_non_exsited_attr(self): estimator = ThreadsafeWatermarkEstimator(ManualWatermarkEstimator(None)) with self.assertRaises(AttributeError): estimator.non_existed_call()
def test_track_timestamp(self): estimator = ThreadsafeWatermarkEstimator(ManualWatermarkEstimator(None)) estimator.observe_timestamp(timestamp.Timestamp(10)) self.assertIsNone(estimator.current_watermark()) estimator.set_watermark(timestamp.Timestamp(20)) self.assertEqual(estimator.current_watermark(), timestamp.Timestamp(20))
def test_get_estimator_state(self): estimator = ThreadsafeWatermarkEstimator(ManualWatermarkEstimator(None)) self.assertIsNone(estimator.get_estimator_state()) estimator.set_watermark(timestamp.Timestamp(10)) self.assertEqual(estimator.get_estimator_state(), timestamp.Timestamp(10))
class PerWindowInvokerSplitTest(unittest.TestCase): def setUp(self): self.window1 = IntervalWindow(0, 10) self.window2 = IntervalWindow(10, 20) self.window3 = IntervalWindow(20, 30) self.windowed_value = WindowedValue( 'a', 57, (self.window1, self.window2, self.window3)) self.restriction = OffsetRange(0, 100) self.watermark_estimator_state = Timestamp(21) self.restriction_provider = TestOffsetRestrictionProvider() self.watermark_estimator = ManualWatermarkEstimator(Timestamp(42)) self.maxDiff = None def create_split_in_window(self, offset_index, windows): return ( SplitResultPrimary(primary_value=WindowedValue((('a', ( OffsetRange(0, offset_index), self.watermark_estimator_state)), offset_index), 57, windows)), SplitResultResidual( residual_value=WindowedValue( (('a', (OffsetRange(offset_index, 100), self.watermark_estimator.get_estimator_state())), 100 - offset_index), 57, windows), current_watermark=self.watermark_estimator.current_watermark(), deferred_timestamp=None)) def create_split_across_windows(self, primary_windows, residual_windows): primary = SplitResultPrimary(primary_value=WindowedValue(( ('a', (OffsetRange(0, 100), self.watermark_estimator_state)), 100), 57, primary_windows)) if primary_windows else None residual = SplitResultResidual( residual_value=WindowedValue( (('a', (OffsetRange(0, 100), self.watermark_estimator_state)), 100), 57, residual_windows), current_watermark=None, deferred_timestamp=None) if residual_windows else None return primary, residual def test_non_window_observing_checkpoint(self): # test checkpoint restriction_tracker = OffsetRestrictionTracker(self.restriction) restriction_tracker.try_claim(30) (primaries, residuals, stop_index) = PerWindowInvoker._try_split( 0.0, None, None, self.windowed_value, self.restriction, self.watermark_estimator_state, self.restriction_provider, restriction_tracker, self.watermark_estimator) expected_primary_split, expected_residual_split = ( self.create_split_in_window(31, self.windowed_value.windows)) self.assertEqual([expected_primary_split], primaries) self.assertEqual([expected_residual_split], residuals) # We don't expect the stop index to be set for non window observing splits self.assertIsNone(stop_index) def test_non_window_observing_split(self): restriction_tracker = OffsetRestrictionTracker(self.restriction) restriction_tracker.try_claim(30) (primaries, residuals, stop_index) = PerWindowInvoker._try_split( 0.1, None, None, self.windowed_value, self.restriction, self.watermark_estimator_state, self.restriction_provider, restriction_tracker, self.watermark_estimator) expected_primary_split, expected_residual_split = ( self.create_split_in_window(37, self.windowed_value.windows)) self.assertEqual([expected_primary_split], primaries) self.assertEqual([expected_residual_split], residuals) # We don't expect the stop index to be set for non window observing splits self.assertIsNone(stop_index) def test_non_window_observing_split_when_restriction_is_done(self): restriction_tracker = OffsetRestrictionTracker(self.restriction) restriction_tracker.try_claim(100) self.assertIsNone( PerWindowInvoker._try_split(0.1, None, None, self.windowed_value, self.restriction, self.watermark_estimator_state, self.restriction_provider, restriction_tracker, self.watermark_estimator)) def test_window_observing_checkpoint_on_first_window(self): restriction_tracker = OffsetRestrictionTracker(self.restriction) restriction_tracker.try_claim(30) (primaries, residuals, stop_index) = PerWindowInvoker._try_split( 0.0, 0, 3, self.windowed_value, self.restriction, self.watermark_estimator_state, self.restriction_provider, restriction_tracker, self.watermark_estimator) expected_primary_split, expected_residual_split = ( self.create_split_in_window(31, (self.window1, ))) _, expected_residual_windows = (self.create_split_across_windows( None, ( self.window2, self.window3, ))) hc.assert_that(primaries, hc.contains_inanyorder(expected_primary_split)) hc.assert_that( residuals, hc.contains_inanyorder( expected_residual_split, expected_residual_windows, )) self.assertEqual(stop_index, 1) def test_window_observing_checkpoint_on_first_window_after_prior_split( self): restriction_tracker = OffsetRestrictionTracker(self.restriction) restriction_tracker.try_claim(30) (primaries, residuals, stop_index) = PerWindowInvoker._try_split( 0.0, 0, 2, # stop index < len(windows) representing a prior split had occurred self.windowed_value, self.restriction, self.watermark_estimator_state, self.restriction_provider, restriction_tracker, self.watermark_estimator) expected_primary_split, expected_residual_split = ( self.create_split_in_window(31, (self.window1, ))) _, expected_residual_windows = (self.create_split_across_windows( None, (self.window2, ))) hc.assert_that(primaries, hc.contains_inanyorder(expected_primary_split)) hc.assert_that( residuals, hc.contains_inanyorder( expected_residual_split, expected_residual_windows, )) self.assertEqual(stop_index, 1) def test_window_observing_split_on_first_window(self): restriction_tracker = OffsetRestrictionTracker(self.restriction) restriction_tracker.try_claim(30) (primaries, residuals, stop_index) = PerWindowInvoker._try_split( 0.2, 0, 3, self.windowed_value, self.restriction, self.watermark_estimator_state, self.restriction_provider, restriction_tracker, self.watermark_estimator) # 20% of 2.7 windows = 20% of 270 offset left = 54 offset # 30 + 54 = 84 split offset expected_primary_split, expected_residual_split = ( self.create_split_in_window(84, (self.window1, ))) _, expected_residual_windows = (self.create_split_across_windows( None, ( self.window2, self.window3, ))) hc.assert_that(primaries, hc.contains_inanyorder(expected_primary_split)) hc.assert_that( residuals, hc.contains_inanyorder( expected_residual_split, expected_residual_windows, )) self.assertEqual(stop_index, 1) def test_window_observing_split_on_middle_window(self): restriction_tracker = OffsetRestrictionTracker(self.restriction) restriction_tracker.try_claim(30) (primaries, residuals, stop_index) = PerWindowInvoker._try_split( 0.2, 1, 3, self.windowed_value, self.restriction, self.watermark_estimator_state, self.restriction_provider, restriction_tracker, self.watermark_estimator) # 20% of 1.7 windows = 20% of 170 offset left = 34 offset # 30 + 34 = 64 split offset expected_primary_split, expected_residual_split = ( self.create_split_in_window(64, (self.window2, ))) expected_primary_windows, expected_residual_windows = ( self.create_split_across_windows((self.window1, ), (self.window3, ))) hc.assert_that( primaries, hc.contains_inanyorder( expected_primary_split, expected_primary_windows, )) hc.assert_that( residuals, hc.contains_inanyorder( expected_residual_split, expected_residual_windows, )) self.assertEqual(stop_index, 2) def test_window_observing_split_on_last_window(self): restriction_tracker = OffsetRestrictionTracker(self.restriction) restriction_tracker.try_claim(30) (primaries, residuals, stop_index) = PerWindowInvoker._try_split( 0.2, 2, 3, self.windowed_value, self.restriction, self.watermark_estimator_state, self.restriction_provider, restriction_tracker, self.watermark_estimator) # 20% of 0.7 windows = 20% of 70 offset left = 14 offset # 30 + 14 = 44 split offset expected_primary_split, expected_residual_split = ( self.create_split_in_window(44, (self.window3, ))) expected_primary_windows, _ = (self.create_split_across_windows(( self.window1, self.window2, ), None)) hc.assert_that( primaries, hc.contains_inanyorder( expected_primary_split, expected_primary_windows, )) hc.assert_that(residuals, hc.contains_inanyorder(expected_residual_split, )) self.assertEqual(stop_index, 3) def test_window_observing_split_on_first_window_fallback(self): restriction_tracker = OffsetRestrictionTracker(self.restriction) restriction_tracker.try_claim(100) # We assume that we can't split this fully claimed restriction self.assertIsNone(restriction_tracker.try_split(0)) (primaries, residuals, stop_index) = PerWindowInvoker._try_split( 0.0, 0, 3, self.windowed_value, self.restriction, self.watermark_estimator_state, self.restriction_provider, restriction_tracker, self.watermark_estimator) expected_primary_windows, expected_residual_windows = ( self.create_split_across_windows((self.window1, ), ( self.window2, self.window3, ))) hc.assert_that(primaries, hc.contains_inanyorder(expected_primary_windows, )) hc.assert_that(residuals, hc.contains_inanyorder(expected_residual_windows, )) self.assertEqual(stop_index, 1) def test_window_observing_split_on_middle_window_fallback(self): restriction_tracker = OffsetRestrictionTracker(self.restriction) restriction_tracker.try_claim(100) # We assume that we can't split this fully claimed restriction self.assertIsNone(restriction_tracker.try_split(0)) (primaries, residuals, stop_index) = PerWindowInvoker._try_split( 0.0, 1, 3, self.windowed_value, self.restriction, self.watermark_estimator_state, self.restriction_provider, restriction_tracker, self.watermark_estimator) expected_primary_windows, expected_residual_windows = ( self.create_split_across_windows(( self.window1, self.window2, ), (self.window3, ))) hc.assert_that(primaries, hc.contains_inanyorder(expected_primary_windows, )) hc.assert_that(residuals, hc.contains_inanyorder(expected_residual_windows, )) self.assertEqual(stop_index, 2) def test_window_observing_split_on_last_window_when_split_not_possible( self): restriction_tracker = OffsetRestrictionTracker(self.restriction) restriction_tracker.try_claim(100) # We assume that we can't split this fully claimed restriction self.assertIsNone(restriction_tracker.try_split(0)) self.assertIsNone( PerWindowInvoker._try_split(0.0, 2, 3, self.windowed_value, self.restriction, self.watermark_estimator_state, self.restriction_provider, restriction_tracker, self.watermark_estimator)) def test_window_observing_split_on_window_boundary_round_up(self): restriction_tracker = OffsetRestrictionTracker(self.restriction) restriction_tracker.try_claim(30) (primaries, residuals, stop_index) = PerWindowInvoker._try_split( 0.6, 0, 3, self.windowed_value, self.restriction, self.watermark_estimator_state, self.restriction_provider, restriction_tracker, self.watermark_estimator) # 60% of 2.7 windows = 60% of 270 offset left = 162 offset # 30 + 162 = 192 offset --> round to end of window 2 expected_primary_windows, expected_residual_windows = ( self.create_split_across_windows(( self.window1, self.window2, ), (self.window3, ))) hc.assert_that(primaries, hc.contains_inanyorder(expected_primary_windows, )) hc.assert_that(residuals, hc.contains_inanyorder(expected_residual_windows, )) self.assertEqual(stop_index, 2) def test_window_observing_split_on_window_boundary_round_down(self): restriction_tracker = OffsetRestrictionTracker(self.restriction) restriction_tracker.try_claim(30) (primaries, residuals, stop_index) = PerWindowInvoker._try_split( 0.3, 0, 3, self.windowed_value, self.restriction, self.watermark_estimator_state, self.restriction_provider, restriction_tracker, self.watermark_estimator) # 30% of 2.7 windows = 30% of 270 offset left = 81 offset # 30 + 81 = 111 offset --> round to end of window 1 expected_primary_windows, expected_residual_windows = ( self.create_split_across_windows((self.window1, ), ( self.window2, self.window3, ))) hc.assert_that(primaries, hc.contains_inanyorder(expected_primary_windows, )) hc.assert_that(residuals, hc.contains_inanyorder(expected_residual_windows, )) self.assertEqual(stop_index, 1) def test_window_observing_split_on_window_boundary_round_down_on_last_window( self): restriction_tracker = OffsetRestrictionTracker(self.restriction) restriction_tracker.try_claim(30) (primaries, residuals, stop_index) = PerWindowInvoker._try_split( 0.9, 0, 3, self.windowed_value, self.restriction, self.watermark_estimator_state, self.restriction_provider, restriction_tracker, self.watermark_estimator) # 90% of 2.7 windows = 90% of 270 offset left = 243 offset # 30 + 243 = 273 offset --> prefer a split so round to end of window 2 # instead of no split expected_primary_windows, expected_residual_windows = ( self.create_split_across_windows(( self.window1, self.window2, ), (self.window3, ))) hc.assert_that(primaries, hc.contains_inanyorder(expected_primary_windows, )) hc.assert_that(residuals, hc.contains_inanyorder(expected_residual_windows, )) self.assertEqual(stop_index, 2)