class SimpleTestStatefulDoFn(DoFn): BUFFER_STATE = CombiningValueStateSpec('buffer', ListCoder(VarIntCoder()), ToListCombineFn()) EXPIRY_TIMER = TimerSpec('expiry1', TimeDomain.WATERMARK) def process(self, element, buffer=DoFn.StateParam(BUFFER_STATE), timer1=DoFn.TimerParam(EXPIRY_TIMER)): unused_key, value = element buffer.add(value) timer1.set(20) @on_timer(EXPIRY_TIMER) def expiry_callback(self, buffer=DoFn.StateParam(BUFFER_STATE), timer=DoFn.TimerParam(EXPIRY_TIMER)): yield ''.join(str(x) for x in sorted(buffer.read()))
def _pardo_group_into_batches(batch_size, input_coder): ELEMENT_STATE = BagStateSpec('values', input_coder) COUNT_STATE = CombiningValueStateSpec('count', input_coder, CountCombineFn()) EXPIRY_TIMER = TimerSpec('expiry', TimeDomain.WATERMARK) class _GroupIntoBatchesDoFn(DoFn): def process(self, element, window=DoFn.WindowParam, element_state=DoFn.StateParam(ELEMENT_STATE), count_state=DoFn.StateParam(COUNT_STATE), expiry_timer=DoFn.TimerParam(EXPIRY_TIMER)): # Allowed lateness not supported in Python SDK # https://beam.apache.org/documentation/programming-guide/#watermarks-and-late-data expiry_timer.set(window.end) element_state.add(element) count_state.add(1) count = count_state.read() if count >= batch_size: batch = [element for element in element_state.read()] key, _ = batch[0] batch_values = [v for (k, v) in batch] yield (key, batch_values) element_state.clear() count_state.clear() @on_timer(EXPIRY_TIMER) def expiry(self, element_state=DoFn.StateParam(ELEMENT_STATE), count_state=DoFn.StateParam(COUNT_STATE)): batch = [element for element in element_state.read()] if batch: key, _ = batch[0] batch_values = [v for (k, v) in batch] yield (key, batch_values) element_state.clear() count_state.clear() return _GroupIntoBatchesDoFn()
class _RemoveOldDuplicates(beam.DoFn): """Internal DoFn that filters out filenames already seen and timestamp unchanged.""" TIME_STATE = CombiningValueStateSpec('count', combine_fn=partial(max, default=0.0)) def process( self, element: Tuple[str, filesystem.FileMetadata], time_state=beam.DoFn.StateParam(TIME_STATE) ) -> Iterable[filesystem.FileMetadata]: path = element[0] file_metadata = element[1] new_ts = file_metadata.last_updated_in_seconds old_ts = time_state.read() if old_ts < new_ts: time_state.add(new_ts) _LOGGER.debug('Generated entry for file %s', path) yield file_metadata else: _LOGGER.debug('File %s was already read', path)
class _RemoveDuplicates(beam.DoFn): """Internal DoFn that filters out filenames already seen (even though the file has updated).""" COUNT_STATE = CombiningValueStateSpec('count', combine_fn=sum) def process( self, element: Tuple[str, filesystem.FileMetadata], count_state=beam.DoFn.StateParam(COUNT_STATE) ) -> Iterable[filesystem.FileMetadata]: path = element[0] file_metadata = element[1] counter = count_state.read() if counter == 0: count_state.add(1) _LOGGER.debug('Generated entry for file %s', path) yield file_metadata else: _LOGGER.debug('File %s was already read, seen %d times', path, counter)
def _pardo_group_into_batches(input_coder, batch_size, max_buffering_duration_secs, clock=time.time): ELEMENT_STATE = BagStateSpec('values', input_coder) COUNT_STATE = CombiningValueStateSpec('count', input_coder, CountCombineFn()) WINDOW_TIMER = TimerSpec('window_end', TimeDomain.WATERMARK) BUFFERING_TIMER = TimerSpec('buffering_end', TimeDomain.REAL_TIME) class _GroupIntoBatchesDoFn(DoFn): def process(self, element, window=DoFn.WindowParam, element_state=DoFn.StateParam(ELEMENT_STATE), count_state=DoFn.StateParam(COUNT_STATE), window_timer=DoFn.TimerParam(WINDOW_TIMER), buffering_timer=DoFn.TimerParam(BUFFERING_TIMER)): # Allowed lateness not supported in Python SDK # https://beam.apache.org/documentation/programming-guide/#watermarks-and-late-data window_timer.set(window.end) element_state.add(element) count_state.add(1) count = count_state.read() if count == 1 and max_buffering_duration_secs > 0: # This is the first element in batch. Start counting buffering time if a # limit was set. buffering_timer.set(clock() + max_buffering_duration_secs) if count >= batch_size: return self.flush_batch(element_state, count_state, buffering_timer) @on_timer(WINDOW_TIMER) def on_window_timer(self, element_state=DoFn.StateParam(ELEMENT_STATE), count_state=DoFn.StateParam(COUNT_STATE), buffering_timer=DoFn.TimerParam(BUFFERING_TIMER)): return self.flush_batch(element_state, count_state, buffering_timer) @on_timer(BUFFERING_TIMER) def on_buffering_timer( self, element_state=DoFn.StateParam(ELEMENT_STATE), count_state=DoFn.StateParam(COUNT_STATE), buffering_timer=DoFn.TimerParam(BUFFERING_TIMER)): return self.flush_batch(element_state, count_state, buffering_timer) def flush_batch(self, element_state, count_state, buffering_timer): batch = [element for element in element_state.read()] if not batch: return key, _ = batch[0] batch_values = [v for (k, v) in batch] element_state.clear() count_state.clear() buffering_timer.clear() yield key, batch_values return _GroupIntoBatchesDoFn()
class GeneralTriggerManagerDoFn(DoFn): """A trigger manager that supports all windowing / triggering cases. This implements a DoFn that manages triggering in a per-key basis. All elements for a single key are processed together. Per-key state holds data related to all windows. """ # TODO(BEAM-12026) Add support for Global and custom window fns. KNOWN_WINDOWS = SetStateSpec('known_windows', IntervalWindowCoder()) FINISHED_WINDOWS = SetStateSpec('finished_windows', IntervalWindowCoder()) LAST_KNOWN_TIME = CombiningValueStateSpec('last_known_time', combine_fn=max) LAST_KNOWN_WATERMARK = CombiningValueStateSpec('last_known_watermark', combine_fn=max) # TODO(pabloem) What's the coder for the elements/keys here? WINDOW_ELEMENT_PAIRS = BagStateSpec( 'all_elements', TupleCoder([IntervalWindowCoder(), PickleCoder()])) WINDOW_TAG_VALUES = BagStateSpec( 'per_window_per_tag_value_state', TupleCoder([IntervalWindowCoder(), StrUtf8Coder(), VarIntCoder()])) PROCESSING_TIME_TIMER = TimerSpec('processing_time_timer', TimeDomain.REAL_TIME) WATERMARK_TIMER = TimerSpec('watermark_timer', TimeDomain.WATERMARK) def __init__(self, windowing: Windowing): self.windowing = windowing # Only session windows are merging. Other windows are non-merging. self.merging_windows = self.windowing.windowfn.is_merging() def process( self, element: typing.Tuple[ K, typing.Iterable[windowed_value.WindowedValue]], all_elements: BagRuntimeState = DoFn.StateParam( WINDOW_ELEMENT_PAIRS), # type: ignore latest_processing_time: AccumulatingRuntimeState = DoFn.StateParam( LAST_KNOWN_TIME), # type: ignore latest_watermark: AccumulatingRuntimeState = DoFn. StateParam( # type: ignore LAST_KNOWN_WATERMARK), window_tag_values: BagRuntimeState = DoFn.StateParam( WINDOW_TAG_VALUES), # type: ignore windows_state: SetRuntimeState = DoFn.StateParam( KNOWN_WINDOWS), # type: ignore finished_windows_state: SetRuntimeState = DoFn. StateParam( # type: ignore FINISHED_WINDOWS), processing_time_timer=DoFn.TimerParam(PROCESSING_TIME_TIMER), watermark_timer=DoFn.TimerParam(WATERMARK_TIMER), *args, **kwargs): context = FnRunnerStatefulTriggerContext( processing_time_timer=processing_time_timer, watermark_timer=watermark_timer, latest_processing_time=latest_processing_time, latest_watermark=latest_watermark, all_elements_state=all_elements, window_tag_values=window_tag_values, finished_windows_state=finished_windows_state) key, windowed_values = element watermark = read_watermark(latest_watermark) windows_to_elements = collections.defaultdict(list) for wv in windowed_values: for window in wv.windows: # ignore expired windows if watermark > window.end + self.windowing.allowed_lateness: continue if window in finished_windows_state.read(): continue windows_to_elements[window].append( TimestampedValue(wv.value, wv.timestamp)) # Processing merging of windows if self.merging_windows: old_windows = set(windows_state.read()) all_windows = old_windows.union(list(windows_to_elements)) if all_windows != old_windows: merge_context = TriggerMergeContext(all_windows, context, self.windowing) self.windowing.windowfn.merge(merge_context) merged_windows_to_elements = collections.defaultdict(list) for window, values in windows_to_elements.items(): while window in merge_context.merged_away: window = merge_context.merged_away[window] merged_windows_to_elements[window].extend(values) windows_to_elements = merged_windows_to_elements for w in windows_to_elements: windows_state.add(w) # Done processing merging of windows seen_windows = set() for w in windows_to_elements: window_context = context.for_window(w) seen_windows.add(w) for value_w_timestamp in windows_to_elements[w]: _LOGGER.debug(value_w_timestamp) all_elements.add((w, value_w_timestamp)) self.windowing.triggerfn.on_element(windowed_values, w, window_context) return self._fire_eligible_windows(key, TimeDomain.WATERMARK, watermark, None, context, seen_windows) def _fire_eligible_windows(self, key: K, time_domain, timestamp: Timestamp, timer_tag: typing.Optional[str], context: 'FnRunnerStatefulTriggerContext', windows_of_interest: typing.Optional[ typing.Set[BoundedWindow]] = None): windows_to_elements = context.windows_to_elements_map() context.all_elements_state.clear() fired_windows = set() _LOGGER.debug('%s - tag %s - timestamp %s', time_domain, timer_tag, timestamp) for w, elems in windows_to_elements.items(): if windows_of_interest is not None and w not in windows_of_interest: # windows_of_interest=None means that we care about all windows. # If we care only about some windows, and this window is not one of # them, then we do not intend to fire this window. continue window_context = context.for_window(w) if self.windowing.triggerfn.should_fire(time_domain, timestamp, w, window_context): finished = self.windowing.triggerfn.on_fire( timestamp, w, window_context) _LOGGER.debug('Firing on window %s. Finished: %s', w, finished) fired_windows.add(w) if finished: context.finished_windows_state.add(w) # TODO(pabloem): Format the output: e.g. pane info elems = [ WindowedValue(e.value, e.timestamp, (w, )) for e in elems ] yield (key, elems) finished_windows: typing.Set[BoundedWindow] = set( context.finished_windows_state.read()) # Add elements that were not fired back into state. for w, elems in windows_to_elements.items(): for e in elems: if (w in finished_windows or (w in fired_windows and self.windowing.accumulation_mode == AccumulationMode.DISCARDING)): continue context.all_elements_state.add((w, e)) @on_timer(PROCESSING_TIME_TIMER) def processing_time_trigger( self, key=DoFn.KeyParam, timer_tag=DoFn.DynamicTimerTagParam, timestamp=DoFn.TimestampParam, latest_processing_time=DoFn.StateParam(LAST_KNOWN_TIME), all_elements=DoFn.StateParam(WINDOW_ELEMENT_PAIRS), processing_time_timer=DoFn.TimerParam(PROCESSING_TIME_TIMER), window_tag_values: BagRuntimeState = DoFn.StateParam( WINDOW_TAG_VALUES), # type: ignore finished_windows_state: SetRuntimeState = DoFn. StateParam( # type: ignore FINISHED_WINDOWS), watermark_timer=DoFn.TimerParam(WATERMARK_TIMER)): context = FnRunnerStatefulTriggerContext( processing_time_timer=processing_time_timer, watermark_timer=watermark_timer, latest_processing_time=latest_processing_time, latest_watermark=None, all_elements_state=all_elements, window_tag_values=window_tag_values, finished_windows_state=finished_windows_state) result = self._fire_eligible_windows(key, TimeDomain.REAL_TIME, timestamp, timer_tag, context) latest_processing_time.add(timestamp) return result @on_timer(WATERMARK_TIMER) def watermark_trigger( self, key=DoFn.KeyParam, timer_tag=DoFn.DynamicTimerTagParam, timestamp=DoFn.TimestampParam, latest_watermark=DoFn.StateParam(LAST_KNOWN_WATERMARK), all_elements=DoFn.StateParam(WINDOW_ELEMENT_PAIRS), processing_time_timer=DoFn.TimerParam(PROCESSING_TIME_TIMER), window_tag_values: BagRuntimeState = DoFn.StateParam( WINDOW_TAG_VALUES), # type: ignore finished_windows_state: SetRuntimeState = DoFn. StateParam( # type: ignore FINISHED_WINDOWS), watermark_timer=DoFn.TimerParam(WATERMARK_TIMER)): context = FnRunnerStatefulTriggerContext( processing_time_timer=processing_time_timer, watermark_timer=watermark_timer, latest_processing_time=None, latest_watermark=latest_watermark, all_elements_state=all_elements, window_tag_values=window_tag_values, finished_windows_state=finished_windows_state) result = self._fire_eligible_windows(key, TimeDomain.WATERMARK, timestamp, timer_tag, context) latest_watermark.add(timestamp) return result
def _pardo_group_into_batches_with_multi_bags( input_coder, batch_size, max_buffering_duration_secs, clock=time.time): ELEMENT_STATE_0 = BagStateSpec('values0', input_coder) ELEMENT_STATE_1 = BagStateSpec('values1', input_coder) ELEMENT_STATE_2 = BagStateSpec('values2', input_coder) ELEMENT_STATE_3 = BagStateSpec('values3', input_coder) COUNT_STATE = CombiningValueStateSpec('count', input_coder, CountCombineFn()) WINDOW_TIMER = TimerSpec('window_end', TimeDomain.WATERMARK) BUFFERING_TIMER = TimerSpec('buffering_end', TimeDomain.REAL_TIME) class _GroupIntoBatchesDoFnWithMultiBags(DoFn): def process( self, element, window=DoFn.WindowParam, element_state_0=DoFn.StateParam(ELEMENT_STATE_0), element_state_1=DoFn.StateParam(ELEMENT_STATE_1), element_state_2=DoFn.StateParam(ELEMENT_STATE_2), element_state_3=DoFn.StateParam(ELEMENT_STATE_3), count_state=DoFn.StateParam(COUNT_STATE), window_timer=DoFn.TimerParam(WINDOW_TIMER), buffering_timer=DoFn.TimerParam(BUFFERING_TIMER)): # Allowed lateness not supported in Python SDK # https://beam.apache.org/documentation/programming-guide/#watermarks-and-late-data window_timer.set(window.end) count_state.add(1) count = count_state.read() element_states = [element_state_0, element_state_1, element_state_2, element_state_3] element_states[count % 4].add(element) if count == 1 and max_buffering_duration_secs > 0: # This is the first element in batch. Start counting buffering time if a # limit was set. buffering_timer.set(clock() + max_buffering_duration_secs) if count >= batch_size: return self.flush_batch(element_states, count_state, buffering_timer) @on_timer(WINDOW_TIMER) def on_window_timer( self, element_state_0=DoFn.StateParam(ELEMENT_STATE_0), element_state_1=DoFn.StateParam(ELEMENT_STATE_1), element_state_2=DoFn.StateParam(ELEMENT_STATE_2), element_state_3=DoFn.StateParam(ELEMENT_STATE_3), count_state=DoFn.StateParam(COUNT_STATE), buffering_timer=DoFn.TimerParam(BUFFERING_TIMER)): element_states = [element_state_0, element_state_1, element_state_2, element_state_3] return self.flush_batch(element_states, count_state, buffering_timer) @on_timer(BUFFERING_TIMER) def on_buffering_timer( self, element_state_0=DoFn.StateParam(ELEMENT_STATE_0), element_state_1=DoFn.StateParam(ELEMENT_STATE_1), element_state_2=DoFn.StateParam(ELEMENT_STATE_2), element_state_3=DoFn.StateParam(ELEMENT_STATE_3), count_state=DoFn.StateParam(COUNT_STATE), buffering_timer=DoFn.TimerParam(BUFFERING_TIMER)): element_states = [element_state_0, element_state_1, element_state_2, element_state_3] return self.flush_batch(element_states, count_state, buffering_timer) def flush_batch(self, element_states, count_state, buffering_timer): batch_values = [] for element_state in element_states: for k, v in element_state.read(): key = k batch_values.append(v) element_state.clear() count_state.clear() buffering_timer.clear() if not batch_values: return yield key, batch_values return _GroupIntoBatchesDoFnWithMultiBags()