def test_windowing_encoding(self): for windowing in ( Windowing(GlobalWindows()), Windowing(FixedWindows(1, 3), AfterCount(6), accumulation_mode=AccumulationMode.ACCUMULATING), Windowing(SlidingWindows(10, 15, 21), AfterCount(28), timestamp_combiner=TimestampCombiner.OUTPUT_AT_LATEST, accumulation_mode=AccumulationMode.DISCARDING)): context = pipeline_context.PipelineContext() self.assertEqual( windowing, Windowing.from_runner_api(windowing.to_runner_api(context), context))
def _execute( self, window_fn, trigger_fn, accumulation_mode, timestamp_combiner, transcript, unused_spec): driver = GeneralTriggerDriver( Windowing(window_fn, trigger_fn, accumulation_mode, timestamp_combiner), TestClock()) state = InMemoryUnmergedState() output = [] watermark = MIN_TIMESTAMP def fire_timers(): to_fire = state.get_and_clear_timers(watermark) while to_fire: for timer_window, (name, time_domain, t_timestamp) in to_fire: for wvalue in driver.process_timer( timer_window, name, time_domain, t_timestamp, state): output.append(_windowed_value_info(wvalue)) to_fire = state.get_and_clear_timers(watermark) for action, params in transcript: if action != 'expect': # Fail if we have output that was not expected in the transcript. self.assertEqual( [], output, msg='Unexpected output: %s before %s: %s' % ( output, action, params)) if action == 'input': bundle = [ WindowedValue(t, t, window_fn.assign(WindowFn.AssignContext(t, t))) for t in params] output = [ _windowed_value_info(wv) for wv in driver.process_elements(state, bundle, watermark)] fire_timers() elif action == 'watermark': watermark = params fire_timers() elif action == 'expect': for expected_output in params: for candidate in output: if all(candidate[k] == expected_output[k] for k in candidate if k in expected_output): output.remove(candidate) break else: self.fail('Unmatched output %s in %s' % (expected_output, output)) elif action == 'state': # TODO(robertwb): Implement once we support allowed lateness. pass else: self.fail('Unknown action: ' + action) # Fail if we have output that was not expected in the transcript. self.assertEqual([], output, msg='Unexpected output: %s' % output)
def test_fixed_after_count_accumulating(self): # yapf: disable test_stream = ( TestStream() .advance_watermark_to(0) .add_elements([('k1', 1), ('k1', 1), ('k2', 1), ('k2', 1)]) .add_elements([('k1', 1), ('k1', 1)]) .advance_watermark_to(2) .add_elements([('k1', 2), ('k2', 2)]) # This values are discarded. .advance_watermark_to_infinity()) # yapf: enable # Fixed, one-second windows with DefaultTrigger (after watermark) windowing = Windowing(FixedWindows(2), triggerfn=Repeatedly(AfterCount(2)), accumulation_mode=AccumulationMode.ACCUMULATING) with TestPipeline() as p: result = ( p | test_stream | WindowInto(windowing.windowfn) | ParDo(trigger_manager._ReifyWindows()) | ParDo(trigger_manager._GroupBundlesByKey()) | ParDo(trigger_manager.GeneralTriggerManagerDoFn(windowing)) | Map(lambda elm: (elm[0], elm[1][0].windows[0], [v.value for v in elm[1]]))) assert_that( result, equal_to([ ('k1', IntervalWindow(0, 2), [1, 1]), ('k2', IntervalWindow(0, 2), [1, 1]), ('k1', IntervalWindow(0, 2), [1, 1, 1, 1]), ]))
def run_trigger(self, window_fn, trigger_fn, accumulation_mode, bundles, late_bundles, expected_panes): actual_panes = collections.defaultdict(list) driver = GeneralTriggerDriver( Windowing(window_fn, trigger_fn, accumulation_mode)) state = InMemoryUnmergedState() for bundle in bundles: for wvalue in driver.process_elements(state, bundle, MIN_TIMESTAMP): window, = wvalue.windows actual_panes[window].append(set(wvalue.value)) while state.timers: for timer_window, (name, time_domain, timestamp) in ( state.get_and_clear_timers()): for wvalue in driver.process_timer( timer_window, name, time_domain, timestamp, state): window, = wvalue.windows actual_panes[window].append(set(wvalue.value)) for bundle in late_bundles: for wvalue in driver.process_elements(state, bundle, MIN_TIMESTAMP): window, = wvalue.windows actual_panes[window].append(set(wvalue.value)) while state.timers: for timer_window, (name, time_domain, timestamp) in ( state.get_and_clear_timers()): for wvalue in driver.process_timer( timer_window, name, time_domain, timestamp, state): window, = wvalue.windows actual_panes[window].append(set(wvalue.value)) self.assertEqual(expected_panes, actual_panes)
def test_serialize_windowing_strategy(self): # This just tests the basic path; more complete tests # are in window_test.py. strategy = Windowing(window.FixedWindows(10)) self.assertEqual( strategy, DataflowRunner.deserialize_windowing_strategy( DataflowRunner.serialize_windowing_strategy(strategy, None)))
def expand(self, pcoll): windowing_saved = pcoll.windowing if windowing_saved.is_default(): # In this (common) case we can use a trivial trigger driver # and avoid the (expensive) window param. globally_windowed = window.GlobalWindows.windowed_value(None) MIN_TIMESTAMP = window.MIN_TIMESTAMP def reify_timestamps(element, timestamp=DoFn.TimestampParam): key, value = element if timestamp == MIN_TIMESTAMP: timestamp = None return key, (value, timestamp) def restore_timestamps(element): key, values = element return [ globally_windowed.with_value((key, value)) if timestamp is None else window.GlobalWindows.windowed_value( (key, value), timestamp) for (value, timestamp) in values ] else: # typing: All conditional function variants must have identical signatures def reify_timestamps( # type: ignore[misc] element, timestamp=DoFn.TimestampParam, window=DoFn.WindowParam): key, value = element # Transport the window as part of the value and restore it later. return key, windowed_value.WindowedValue( value, timestamp, [window]) def restore_timestamps(element): key, windowed_values = element return [ wv.with_value((key, wv.value)) for wv in windowed_values ] ungrouped = pcoll | Map(reify_timestamps).with_output_types(Any) # TODO(BEAM-8104) Using global window as one of the standard window. # This is to mitigate the Dataflow Java Runner Harness limitation to # accept only standard coders. ungrouped._windowing = Windowing( window.GlobalWindows(), triggerfn=Always(), accumulation_mode=AccumulationMode.DISCARDING, timestamp_combiner=TimestampCombiner.OUTPUT_AT_EARLIEST) result = (ungrouped | GroupByKey() | FlatMap(restore_timestamps).with_output_types(Any)) result._windowing = windowing_saved return result
def expand(self, pcoll): windowing_saved = pcoll.windowing if windowing_saved.is_default(): # In this (common) case we can use a trivial trigger driver # and avoid the (expensive) window param. globally_windowed = window.GlobalWindows.windowed_value(None) window_fn = window.GlobalWindows() MIN_TIMESTAMP = window.MIN_TIMESTAMP def reify_timestamps(element, timestamp=DoFn.TimestampParam): key, value = element if timestamp == MIN_TIMESTAMP: timestamp = None return key, (value, timestamp) def restore_timestamps(element): key, values = element return [ globally_windowed.with_value((key, value)) if timestamp is None else window.GlobalWindows.windowed_value((key, value), timestamp) for (value, timestamp) in values] else: # The linter is confused. # hash(1) is used to force "runtime" selection of _IdentityWindowFn # pylint: disable=abstract-class-instantiated cls = hash(1) and _IdentityWindowFn window_fn = cls( windowing_saved.windowfn.get_window_coder()) def reify_timestamps(element, timestamp=DoFn.TimestampParam): key, value = element return key, TimestampedValue(value, timestamp) def restore_timestamps(element, window=DoFn.WindowParam): # Pass the current window since _IdentityWindowFn wouldn't know how # to generate it. key, values = element return [ windowed_value.WindowedValue( (key, value.value), value.timestamp, [window]) for value in values] ungrouped = pcoll | Map(reify_timestamps) ungrouped._windowing = Windowing( window_fn, triggerfn=AfterCount(1), accumulation_mode=AccumulationMode.DISCARDING, timestamp_combiner=TimestampCombiner.OUTPUT_AT_EARLIEST) result = (ungrouped | GroupByKey() | FlatMap(restore_timestamps)) result._windowing = windowing_saved return result
def deserialize_windowing_strategy(cls, serialized_data): # Imported here to avoid circular dependencies. # pylint: disable=wrong-import-order, wrong-import-position from apache_beam.runners import pipeline_context from apache_beam.portability.api import beam_runner_api_pb2 from apache_beam.transforms.core import Windowing proto = beam_runner_api_pb2.MessageWithComponents() proto.ParseFromString(cls.json_string_to_byte_array(serialized_data)) return Windowing.from_runner_api( proto.windowing_strategy, pipeline_context.PipelineContext(proto.components))
def get_windowing(self, inputs): # type: (Any) -> Windowing """Returns the window function to be associated with transform's output. By default most transforms just return the windowing function associated with the input PCollection (or the first input if several). """ if inputs: return inputs[0].windowing else: from apache_beam.transforms.core import Windowing from apache_beam.transforms.window import GlobalWindows # TODO(robertwb): Return something compatible with every windowing? return Windowing(GlobalWindows())
def run_trigger(self, window_fn, trigger_fn, accumulation_mode, bundles, late_bundles, expected_panes): actual_panes = collections.defaultdict(list) allowed_lateness = Duration( micros=int(common_urns.constants.MAX_TIMESTAMP_MILLIS.constant) * 1000) driver = GeneralTriggerDriver( Windowing(window_fn, trigger_fn, accumulation_mode, allowed_lateness=allowed_lateness), TestClock()) state = InMemoryUnmergedState() for bundle in bundles: for wvalue in driver.process_elements(state, bundle, MIN_TIMESTAMP, MIN_TIMESTAMP): window, = wvalue.windows self.assertEqual(window.max_timestamp(), wvalue.timestamp) actual_panes[window].append(set(wvalue.value)) while state.timers: for timer_window, (name, time_domain, timestamp) in (state.get_and_clear_timers()): for wvalue in driver.process_timer(timer_window, name, time_domain, timestamp, state, MIN_TIMESTAMP): window, = wvalue.windows self.assertEqual(window.max_timestamp(), wvalue.timestamp) actual_panes[window].append(set(wvalue.value)) for bundle in late_bundles: for wvalue in driver.process_elements(state, bundle, MAX_TIMESTAMP, MAX_TIMESTAMP): window, = wvalue.windows self.assertEqual(window.max_timestamp(), wvalue.timestamp) actual_panes[window].append(set(wvalue.value)) while state.timers: for timer_window, (name, time_domain, timestamp) in ( state.get_and_clear_timers()): for wvalue in driver.process_timer(timer_window, name, time_domain, timestamp, state, MAX_TIMESTAMP): window, = wvalue.windows self.assertEqual(window.max_timestamp(), wvalue.timestamp) actual_panes[window].append(set(wvalue.value)) self.assertEqual(expected_panes, actual_panes)
def create(factory, transform_id, transform_proto, parameter, consumers): class WindowIntoDoFn(beam.DoFn): def __init__(self, windowing): self.windowing = windowing def process(self, element, timestamp=beam.DoFn.TimestampParam): new_windows = self.windowing.windowfn.assign( WindowFn.AssignContext(timestamp, element=element)) yield WindowedValue(element, timestamp, new_windows) from apache_beam.transforms.core import Windowing from apache_beam.transforms.window import WindowFn, WindowedValue windowing = Windowing.from_runner_api(parameter, factory.context) return _create_simple_pardo_operation( factory, transform_id, transform_proto, consumers, WindowIntoDoFn(windowing))
def test_fixed_windows_simple_watermark(self): def tsv(key, value, ts): return TimestampedValue((key, value), timestamp=ts) # yapf: disable test_stream = ( TestStream() .advance_watermark_to(0) .add_elements([tsv('k1', 1, 0), tsv('k2', 1, 0), tsv('k1', 2, 0), tsv('k2', 2, 0)]) .add_elements([tsv('k1', 3, 0), tsv('k2', 3, 0)]) .add_elements([tsv('k1', 4, 1), tsv('k2', 4, 1)]) .add_elements([tsv('k1', 5, 1), tsv('k2', 5, 1)]) .advance_watermark_to(1) .add_elements([tsv('k1', 6, 0)]) .advance_watermark_to_infinity()) # yapf: enable # Fixed, one-second windows with DefaultTrigger (after watermark) windowing = Windowing(FixedWindows(1), allowed_lateness=MAX_TIMESTAMP.seconds()) with TestPipeline() as p: result = ( p | test_stream | WindowInto(windowing.windowfn) | ParDo(trigger_manager._ReifyWindows()) | ParDo(trigger_manager._GroupBundlesByKey()) | ParDo(trigger_manager.GeneralTriggerManagerDoFn(windowing)) | Map(lambda elm: (elm[0], elm[1][0].windows[0], [v.value for v in elm[1]]))) assert_that( result, equal_to([ ('k1', IntervalWindow(0, 1), [1, 2, 3]), # On the watermark ('k2', IntervalWindow(0, 1), [1, 2, 3]), # On the watermark ('k1', IntervalWindow(1, 2), [4, 5]), # On the watermark ('k2', IntervalWindow(1, 2), [4, 5]), # On the watermark ('k1', IntervalWindow(0, 1), [6]), # After the watermark ]))
def test_sessions_and_complex_trigger_accumulating(self): def tsv(key, value, ts): return TimestampedValue((key, value), timestamp=ts) # yapf: disable test_stream = ( TestStream() .advance_watermark_to(0) .add_elements([tsv('k1', 1, 1), tsv('k1', 2, 15), tsv('k1', 3, 7), tsv('k1', 4, 30)]) .advance_watermark_to(50) .add_elements([tsv('k1', -3, 1), tsv('k1', -2, 2),]) .add_elements([tsv('k1', -1, 21)]) .advance_watermark_to_infinity()) # yapf: enable # Fixed, one-second windows with DefaultTrigger (after watermark) windowing = Windowing(Sessions(10), triggerfn=AfterWatermark(early=AfterCount(2), late=AfterCount(1)), accumulation_mode=AccumulationMode.ACCUMULATING, allowed_lateness=MAX_TIMESTAMP.seconds()) with TestPipeline() as p: result = (p | test_stream | WindowInto(windowing.windowfn) | ParDo(trigger_manager._ReifyWindows()) | ParDo(trigger_manager._GroupBundlesByKey()) | ParDo( trigger_manager.GeneralTriggerManagerDoFn(windowing)) | Map(lambda elm: (elm[0], elm[1][0].windows[0], set(v.value for v in elm[1])))) assert_that( result, equal_to([ ('k1', IntervalWindow(1, 25), {1, 2, 3}), # early ('k1', IntervalWindow(1, 25), {1, 2, 3}), # on time ('k1', IntervalWindow(30, 40), {4}), # on time ('k1', IntervalWindow(1, 25), {1, 2, 3, -3, -2}), # late ('k1', IntervalWindow(1, 40), {1, 2, 3, 4, -3, -2, -1}), # late ]))
def test_sliding_windows_simple_watermark(self): # yapf: disable test_stream = ( TestStream() .advance_watermark_to(0) .add_elements([('k1', 1), ('k2', 1), ('k1', 1), ('k2', 1)]) .add_elements([('k1', 1), ('k2', 1)]) .advance_watermark_to(1) .add_elements([('k1', 2), ('k2', 2)]) .add_elements([('k1', 2), ('k2', 2)]) .advance_watermark_to(2) .add_elements([('k1', 3), ('k2', 3)]) .add_elements([('k1', 3), ('k2', 3)]) .advance_watermark_to_infinity()) # yapf: enable # Fixed, one-second windows with DefaultTrigger (after watermark) windowing = Windowing(SlidingWindows(2, 1)) with TestPipeline() as p: result = ( p | test_stream | WindowInto(windowing.windowfn) | ParDo(trigger_manager._ReifyWindows()) | ParDo(trigger_manager._GroupBundlesByKey()) | ParDo(trigger_manager.GeneralTriggerManagerDoFn(windowing)) | Map(lambda elm: (elm[0], elm[1][0].windows[0], [v.value for v in elm[1]]))) assert_that( result, equal_to([ ('k1', IntervalWindow(-1, 1), [1, 1, 1]), ('k2', IntervalWindow(-1, 1), [1, 1, 1]), ('k1', IntervalWindow(0, 2), [1, 1, 1, 2, 2]), ('k2', IntervalWindow(0, 2), [1, 1, 1, 2, 2]), ('k1', IntervalWindow(1, 3), [2, 2, 3, 3]), ('k2', IntervalWindow(1, 3), [2, 2, 3, 3]), ('k1', IntervalWindow(2, 4), [3, 3]), ('k2', IntervalWindow(2, 4), [3, 3]), ]))
def test_with_trigger_window_that_finish(self): def tsv(key, value, ts): return TimestampedValue((key, value), timestamp=ts) # yapf: disable test_stream = ( TestStream() .advance_watermark_to(0) .add_elements([tsv('k1', 1, 0), tsv('k1', 2, 0)]) .add_elements([tsv('k1', 3, 0)]) .advance_watermark_to(2) .add_elements([tsv('k1', 6, 0)]) # This value is discarded. .advance_watermark_to_infinity()) # yapf: enable # Fixed, one-second windows with DefaultTrigger (after watermark) windowing = Windowing(FixedWindows(1), triggerfn=AfterWatermark(), allowed_lateness=0, accumulation_mode=AccumulationMode.DISCARDING) with TestPipeline() as p: result = ( p | test_stream | WindowInto(windowing.windowfn) | ParDo(trigger_manager._ReifyWindows()) | ParDo(trigger_manager._GroupBundlesByKey()) | ParDo(trigger_manager.GeneralTriggerManagerDoFn(windowing)) | Map(lambda elm: (elm[0], elm[1][0].windows[0], [v.value for v in elm[1]]))) assert_that( result, equal_to([ ('k1', IntervalWindow(0, 1), [1, 2, 3]), # On the watermark ]))
def _run_log(self, spec): def parse_int_list(s): """Parses strings like '[1, 2, 3]'.""" s = s.strip() assert s[0] == '[' and s[-1] == ']', s if not s[1:-1].strip(): return [] return [int(x) for x in s[1:-1].split(',')] def split_args(s): """Splits 'a, b, [c, d]' into ['a', 'b', '[c, d]'].""" args = [] start = 0 depth = 0 for ix in range(len(s)): c = s[ix] if c in '({[': depth += 1 elif c in ')}]': depth -= 1 elif c == ',' and depth == 0: args.append(s[start:ix].strip()) start = ix + 1 assert depth == 0, s args.append(s[start:].strip()) return args def parse(s, names): """Parse (recursive) 'Foo(arg, kw=arg)' for Foo in the names dict.""" s = s.strip() if s in names: return names[s] elif s[0] == '[': return parse_int_list(s) elif '(' in s: assert s[-1] == ')', s callee = parse(s[:s.index('(')], names) posargs = [] kwargs = {} for arg in split_args(s[s.index('(') + 1:-1]): if '=' in arg: kw, value = arg.split('=', 1) kwargs[kw] = parse(value, names) else: posargs.append(parse(arg, names)) return callee(*posargs, **kwargs) else: try: return int(s) except ValueError: raise ValueError('Unknown function: %s' % s) def parse_fn(s, names): """Like parse(), but implicitly calls no-arg constructors.""" fn = parse(s, names) if isinstance(fn, type): return fn() return fn # pylint: disable=wrong-import-order, wrong-import-position from apache_beam.transforms import window as window_module # pylint: enable=wrong-import-order, wrong-import-position window_fn_names = dict(window_module.__dict__) window_fn_names.update({ 'CustomTimestampingFixedWindowsWindowFn': CustomTimestampingFixedWindowsWindowFn }) trigger_names = {'Default': DefaultTrigger} trigger_names.update(trigger.__dict__) window_fn = parse_fn(spec.get('window_fn', 'GlobalWindows'), window_fn_names) trigger_fn = parse_fn(spec.get('trigger_fn', 'Default'), trigger_names) accumulation_mode = getattr( AccumulationMode, spec.get('accumulation_mode', 'ACCUMULATING').upper()) timestamp_combiner = getattr( TimestampCombiner, spec.get('timestamp_combiner', 'OUTPUT_AT_EOW').upper()) driver = GeneralTriggerDriver( Windowing(window_fn, trigger_fn, accumulation_mode, timestamp_combiner), TestClock()) state = InMemoryUnmergedState() output = [] watermark = MIN_TIMESTAMP def fire_timers(): to_fire = state.get_and_clear_timers(watermark) while to_fire: for timer_window, (name, time_domain, t_timestamp) in to_fire: for wvalue in driver.process_timer(timer_window, name, time_domain, t_timestamp, state): window, = wvalue.windows output.append({ 'window': [window.start, window.end - 1], 'values': sorted(wvalue.value), 'timestamp': wvalue.timestamp }) to_fire = state.get_and_clear_timers(watermark) for line in spec['transcript']: action, params = line.items()[0] if action != 'expect': # Fail if we have output that was not expected in the transcript. self.assertEquals([], output, msg='Unexpected output: %s before %s' % (output, line)) if action == 'input': bundle = [ WindowedValue( t, t, window_fn.assign(WindowFn.AssignContext(t, t))) for t in params ] output = [{ 'window': [wvalue.windows[0].start, wvalue.windows[0].end - 1], 'values': sorted(wvalue.value), 'timestamp': wvalue.timestamp } for wvalue in driver.process_elements( state, bundle, watermark)] fire_timers() elif action == 'watermark': watermark = params fire_timers() elif action == 'expect': for expected_output in params: for candidate in output: if all(candidate[k] == expected_output[k] for k in candidate if k in expected_output): output.remove(candidate) break else: self.fail('Unmatched output %s in %s' % (expected_output, output)) elif action == 'state': # TODO(robertwb): Implement once we support allowed lateness. pass else: self.fail('Unknown action: ' + action) # Fail if we have output that was not expected in the transcript. self.assertEquals([], output, msg='Unexpected output: %s' % output)