rohdesamuel commented on a change in pull request #10497: [BEAM-8335] Add the ReverseTestStream URL: https://github.com/apache/beam/pull/10497#discussion_r386723371
########## File path: sdks/python/apache_beam/testing/test_stream.py ########## @@ -314,3 +355,239 @@ def from_runner_api_parameter(ptransform, payload, context): coder=coder, events=[Event.from_runner_api(e, coder) for e in payload.events], output_tags=output_tags) + + +class TimingInfo(object): + def __init__(self, processing_time, watermark): + self._processing_time = timestamp.Timestamp.of(processing_time) + self._watermark = timestamp.Timestamp.of(watermark) + + @property + def processing_time(self): + return self._processing_time + + @property + def watermark(self): + return self._watermark + + def __repr__(self): + return '({}, {}, {})'.format( + self.event_timestamp, self.processing_time, self.watermark) + + +class PairWithTiming(PTransform): + """Pairs the input element with timing information. + + Input: element; output: KV(element, timing information) + Where timing information := (processing time, watermark) + + This is used in the ReverseTestStream implementation to replay watermark + advancements. + """ + + URN = "beam:transform:pair_with_timing:v1" + + def expand(self, pcoll): + return pvalue.PCollection.from_(pcoll) + + +class ReverseTestStream(PTransform): + """A Transform that can create TestStream events from a stream of elements. + + This currently assumes that this the pipeline being run on a single machine + and elements come in order and are outputted in the same order that they came + in. + """ + class Format(Enum): + TEST_STREAM_EVENTS = 1 + TEST_STREAM_FILE_RECORDS = 2 + SERIALIZED_TEST_STREAM_FILE_RECORDS = 3 + + def __init__( + self, sample_resolution_sec, output_tag, coder=None, output_format=None): + self._sample_resolution_sec = sample_resolution_sec + self._output_tag = output_tag + self._output_format = output_format if output_format \ + else ReverseTestStream.Format.TEST_STREAM_EVENTS + self._coder = coder if coder else beam.coders.FastPrimitivesCoder() + + def expand(self, pcoll): + generator = ( + _TestStreamFileRecordGenerator(coder=self._coder) if ( + self._output_format in ( + self.Format.TEST_STREAM_FILE_RECORDS, + self.Format.SERIALIZED_TEST_STREAM_FILE_RECORDS)) else + _TestStreamEventGenerator()) + + ret = ( + pcoll + | beam.WindowInto(beam.window.GlobalWindows()) + + # First get the initial timing information. This will be used to start + # the periodic timers which will generate processing time and watermark + # advancements every `sample_resolution_sec`. + | 'initial timing' >> PairWithTiming() + + # Next, map every element to the same key so that only a single timer is + # started for this given ReverseTestStream. + | beam.Map(lambda x: (0, x)) + + # Next, pass-through each element which will be paired with its timing + # info in the next step. Also, start the periodic timers. We use timers + # in this situation to capture watermark advancements that occur when + # there are no elements being produced upstream. + | beam.ParDo( + _WatermarkEventGenerator( + output_tag=self._output_tag, + sample_resolution_sec=self._sample_resolution_sec)) + + # Next, retrieve the timing information for watermark events that were + # generated in the previous step. This is because elements generated + # through the timers don't have their timing information yet. + | 'timing info for watermarks' >> PairWithTiming() + + # Format the events properly. + | beam.ParDo(generator)) + + if self._output_format == self.Format.SERIALIZED_TEST_STREAM_FILE_RECORDS: + + def serializer(e): + return e.SerializeToString() + + ret = ret | 'serializer' >> beam.Map(serializer) + + return ret + + +class _WatermarkEventGenerator(beam.DoFn): + # Used to return the initial timing information. + EXECUTE_ONCE_STATE = beam.transforms.userstate.BagStateSpec( + name='execute_once_state', coder=beam.coders.FastPrimitivesCoder()) + WATERMARK_TRACKER = TimerSpec('watermark_tracker', TimeDomain.REAL_TIME) + + def __init__(self, output_tag, sample_resolution_sec=0.1): + self._output_tag = output_tag + self._sample_resolution_sec = sample_resolution_sec + + @on_timer(WATERMARK_TRACKER) + def on_watermark_tracker( + self, + timestamp=beam.DoFn.TimestampParam, + window=beam.DoFn.WindowParam, + watermark_tracker=beam.DoFn.TimerParam(WATERMARK_TRACKER)): + next_sample_time = (timestamp.micros * 1e-6) + self._sample_resolution_sec + watermark_tracker.set(next_sample_time) + + # Generate two events, the delta since the last sample and a place-holder + # WatermarkEvent. This is a placeholder because we can't otherwise add the + # watermark from the runner to the event. + yield ProcessingTimeEvent(self._sample_resolution_sec) + yield WatermarkEvent(MIN_TIMESTAMP) + + def process( + self, + e, + timestamp=beam.DoFn.TimestampParam, + window=beam.DoFn.WindowParam, + watermark_tracker=beam.DoFn.TimerParam(WATERMARK_TRACKER), + execute_once_state=beam.DoFn.StateParam(EXECUTE_ONCE_STATE)): + + _, (element, timing_info) = e + + first_time = next(execute_once_state.read(), True) + if first_time: + # Generate the initial timing events. + execute_once_state.add(False) + now_sec = timing_info.processing_time.micros * 1e-6 + watermark_tracker.set(now_sec + self._sample_resolution_sec) + + # Here we capture the initial time offset and initial watermark. This is + # where we emit the TestStreamFileHeader. + yield TestStreamFileHeader(tag=self._output_tag) + yield ProcessingTimeEvent( + Duration(micros=timing_info.processing_time.micros)) + yield WatermarkEvent(MIN_TIMESTAMP) + yield element + + +class _TestStreamEventGenerator(beam.DoFn): + def start_bundle(self): + self.elements = [] + self.timing_events = [] + + def finish_bundle(self): + if self.timing_events: + yield WindowedValue( + self.timing_events, timestamp=0, windows=[beam.window.GlobalWindow()]) + + if self.elements: + yield WindowedValue([ElementEvent(self.elements)], + timestamp=0, + windows=[beam.window.GlobalWindow()]) + + def process(self, e, timestamp=beam.DoFn.TimestampParam): + element, timing_info = e + if isinstance(element, WatermarkEvent): + element.new_watermark = timing_info.watermark.micros + self.timing_events.append(element) + elif isinstance(element, ProcessingTimeEvent): + self.timing_events.append(element) Review comment: ack (see previous comment) ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services