def flush(self, stamp=DoFn.TimestampParam, key=DoFn.KeyParam, buffer=DoFn.StateParam(BUFFER), minStamp=DoFn.StateParam(MIN_STAMP), flushTimer=DoFn.TimerParam(FLUSH_TIMER)): keep, flush = [], [] minKeepStamp = None for item in buffer.read(): if item[2] <= stamp: flush.append(item) else: keep.append(item) if not minKeepStamp or minKeepStamp > item[2]: minKeepStamp = item[2] outputs = [] if flush: flush = list(sorted(flush, key=lambda x: x[2])) outputs = list(self.flushMetrics(flush, key)) keep.append(flush[-1]) buffer.clear() for item in keep: buffer.add(item) if minKeepStamp: flushTimer.set(minKeepStamp) minStamp.write(minKeepStamp) else: minStamp.clear() return outputs
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 process(self, element, stamp=DoFn.TimestampParam, buffer=DoFn.StateParam(BUFFER), minStamp=DoFn.StateParam(MIN_STAMP), flushTimer=DoFn.TimerParam(FLUSH_TIMER)): currentMinStamp = minStamp.read() or stamp if currentMinStamp == stamp: minStamp.write(stamp) flushTimer.set(stamp) buffer.add(element[1])
def test_pipeline_sdk_not_overridden(self): pipeline_options = PipelineOptions([ '--experiments=beam_fn_api', '--experiments=use_unified_worker', '--temp_location', 'gs://any-location/temp', '--sdk_container_image=dummy_prefix/dummy_name:dummy_tag' ]) pipeline = Pipeline(options=pipeline_options) pipeline | Create([1, 2, 3]) | ParDo(DoFn()) # pylint:disable=expression-not-assigned proto_pipeline, _ = pipeline.to_runner_api(return_context=True) dummy_env = DockerEnvironment( container_image='dummy_prefix/dummy_name:dummy_tag') proto_pipeline, _ = pipeline.to_runner_api( return_context=True, default_environment=dummy_env) # Accessing non-public method for testing. apiclient.DataflowApplicationClient._apply_sdk_environment_overrides( proto_pipeline, {}, pipeline_options) self.assertIsNotNone(2, len(proto_pipeline.components.environments)) from apache_beam.utils import proto_utils found_override = False for env in proto_pipeline.components.environments.values(): docker_payload = proto_utils.parse_Bytes( env.payload, beam_runner_api_pb2.DockerPayload) if docker_payload.container_image.startswith( names.DATAFLOW_CONTAINER_IMAGE_REPOSITORY): found_override = True self.assertFalse(found_override)
def process(self, element, batch=DoFn.StateParam(BATCH), batchSize=DoFn.StateParam(BATCH_SIZE), flushTimer=DoFn.TimerParam(FLUSH_TIMER), endOfTime=DoFn.TimerParam(EOW_TIMER)): from apache_beam.utils.timestamp import Timestamp, Duration from apache_beam.transforms.window import GlobalWindow currentSize = batchSize.read() if not currentSize: currentSize = 1 flushTimer.set(Timestamp.now() + Duration(micros=self.maxWaitTime * 1000)) endOfTime.set(GlobalWindow().max_timestamp()) else: currentSize += 1 batchSize.write(currentSize) batch.add(element[1]) if currentSize >= self.batchSize: return self.flush(batch, batchSize)
def test_sdk_harness_container_images_get_set(self): pipeline_options = PipelineOptions([ '--experiments=beam_fn_api', '--experiments=use_unified_worker', '--temp_location', 'gs://any-location/temp' ]) pipeline = Pipeline(options=pipeline_options) pipeline | Create([1, 2, 3]) | ParDo(DoFn()) # pylint:disable=expression-not-assigned test_environment = DockerEnvironment( container_image='test_default_image') proto_pipeline, _ = pipeline.to_runner_api( return_context=True, default_environment=test_environment) # We have to manually add environments since Dataflow only sets # 'sdkHarnessContainerImages' when there are at least two environments. dummy_env = beam_runner_api_pb2.Environment( urn=common_urns.environments.DOCKER.urn, payload=(beam_runner_api_pb2.DockerPayload( container_image='dummy_image')).SerializeToString()) proto_pipeline.components.environments['dummy_env_id'].CopyFrom( dummy_env) dummy_transform = beam_runner_api_pb2.PTransform( environment_id='dummy_env_id') proto_pipeline.components.transforms['dummy_transform_id'].CopyFrom( dummy_transform) env = apiclient.Environment( [], # packages pipeline_options, '2.0.0', # any environment version FAKE_PIPELINE_URL, proto_pipeline, _sdk_image_overrides={ '.*dummy.*': 'dummy_image', '.*test.*': 'test_default_image' }) worker_pool = env.proto.workerPools[0] # For the test, a third environment get added since actual default # container image for Dataflow is different from 'test_default_image' # we've provided above. self.assertEqual(3, len(worker_pool.sdkHarnessContainerImages)) # Container image should be overridden by a Dataflow specific URL. self.assertTrue( str.startswith( (worker_pool.sdkHarnessContainerImages[0]).containerImage, 'gcr.io/cloud-dataflow/v1beta3/python'))
def test_java_sdk_harness_dedup(self): pipeline_options = PipelineOptions([ '--experiments=beam_fn_api', '--experiments=use_unified_worker', '--temp_location', 'gs://any-location/temp' ]) pipeline = Pipeline(options=pipeline_options) pipeline | Create([1, 2, 3]) | ParDo(DoFn()) # pylint:disable=expression-not-assigned proto_pipeline, _ = pipeline.to_runner_api(return_context=True) dummy_env_1 = beam_runner_api_pb2.Environment( urn=common_urns.environments.DOCKER.urn, payload=(beam_runner_api_pb2.DockerPayload( container_image='apache/beam_java:dummy_tag') ).SerializeToString()) proto_pipeline.components.environments['dummy_env_id_1'].CopyFrom( dummy_env_1) dummy_transform_1 = beam_runner_api_pb2.PTransform( environment_id='dummy_env_id_1') proto_pipeline.components.transforms['dummy_transform_id_1'].CopyFrom( dummy_transform_1) dummy_env_2 = beam_runner_api_pb2.Environment( urn=common_urns.environments.DOCKER.urn, payload=(beam_runner_api_pb2.DockerPayload( container_image='apache/beam_java:dummy_tag') ).SerializeToString()) proto_pipeline.components.environments['dummy_env_id_2'].CopyFrom( dummy_env_2) dummy_transform_2 = beam_runner_api_pb2.PTransform( environment_id='dummy_env_id_2') proto_pipeline.components.transforms['dummy_transform_id_2'].CopyFrom( dummy_transform_2) # Accessing non-public method for testing. apiclient.DataflowApplicationClient._apply_sdk_environment_overrides( proto_pipeline, dict(), pipeline_options) # Only one of 'dummy_env_id_1' or 'dummy_env_id_2' should be in the set of # environment IDs used by the proto after Java environment de-duping. env_ids_from_transforms = [ proto_pipeline.components.transforms[transform_id].environment_id for transform_id in proto_pipeline.components.transforms ] if 'dummy_env_id_1' in env_ids_from_transforms: self.assertTrue('dummy_env_id_2' not in env_ids_from_transforms) else: self.assertTrue('dummy_env_id_2' in env_ids_from_transforms)
def test_default_environment_get_set(self): pipeline_options = PipelineOptions([ '--experiments=beam_fn_api', '--experiments=use_unified_worker', '--temp_location', 'gs://any-location/temp' ]) pipeline = Pipeline(options=pipeline_options) pipeline | Create([1, 2, 3]) | ParDo(DoFn()) # pylint:disable=expression-not-assigned test_environment = DockerEnvironment(container_image='test_default_image') proto_pipeline, _ = pipeline.to_runner_api( return_context=True, default_environment=test_environment) dummy_env = beam_runner_api_pb2.Environment( urn=common_urns.environments.DOCKER.urn, payload=( beam_runner_api_pb2.DockerPayload( container_image='dummy_image')).SerializeToString()) proto_pipeline.components.environments['dummy_env_id'].CopyFrom(dummy_env) dummy_transform = beam_runner_api_pb2.PTransform( environment_id='dummy_env_id') proto_pipeline.components.transforms['dummy_transform_id'].CopyFrom( dummy_transform) env = apiclient.Environment( [], # packages pipeline_options, '2.0.0', # any environment version FAKE_PIPELINE_URL, proto_pipeline, _sdk_image_overrides={ '.*dummy.*': 'dummy_image', '.*test.*': 'test_default_image' }) worker_pool = env.proto.workerPools[0] self.assertEqual(2, len(worker_pool.sdkHarnessContainerImages)) images_from_proto = [ sdk_info.containerImage for sdk_info in worker_pool.sdkHarnessContainerImages ] self.assertIn('test_default_image', images_from_proto)
def test_non_apache_container_not_overridden(self): pipeline_options = PipelineOptions([ '--experiments=beam_fn_api', '--experiments=use_unified_worker', '--temp_location', 'gs://any-location/temp' ]) pipeline = Pipeline(options=pipeline_options) pipeline | Create([1, 2, 3]) | ParDo(DoFn()) # pylint:disable=expression-not-assigned proto_pipeline, _ = pipeline.to_runner_api(return_context=True) dummy_env = beam_runner_api_pb2.Environment( urn=common_urns.environments.DOCKER.urn, payload=( beam_runner_api_pb2.DockerPayload( container_image='other_org/dummy_name:dummy_tag') ).SerializeToString()) proto_pipeline.components.environments['dummy_env_id'].CopyFrom(dummy_env) dummy_transform = beam_runner_api_pb2.PTransform( environment_id='dummy_env_id') proto_pipeline.components.transforms['dummy_transform_id'].CopyFrom( dummy_transform) # Accessing non-public method for testing. apiclient.DataflowApplicationClient._apply_sdk_environment_overrides( proto_pipeline, dict(), pipeline_options) self.assertIsNotNone(2, len(proto_pipeline.components.environments)) from apache_beam.utils import proto_utils found_override = False for env in proto_pipeline.components.environments.values(): docker_payload = proto_utils.parse_Bytes( env.payload, beam_runner_api_pb2.DockerPayload) if docker_payload.container_image.startswith( names.DATAFLOW_CONTAINER_IMAGE_REPOSITORY): found_override = True self.assertFalse(found_override)
def __init__( self, fn, # type: core.DoFn args, kwargs, side_inputs, # type: Iterable[sideinputs.SideInputMap] windowing, tagged_receivers, # type: Mapping[Optional[str], Receiver] step_name=None, # type: Optional[str] logging_context=None, state=None, scoped_metrics_container=None, operation_name=None, user_state_context=None # type: Optional[userstate.UserStateContext] ): """Initializes a DoFnRunner. Args: fn: user DoFn to invoke args: positional side input arguments (static and placeholder), if any kwargs: keyword side input arguments (static and placeholder), if any side_inputs: list of sideinput.SideInputMaps for deferred side inputs windowing: windowing properties of the output PCollection(s) tagged_receivers: a dict of tag name to Receiver objects step_name: the name of this step logging_context: DEPRECATED [BEAM-4728] state: handle for accessing DoFn state scoped_metrics_container: DEPRECATED operation_name: The system name assigned by the runner for this operation. user_state_context: The UserStateContext instance for the current Stateful DoFn. """ # Need to support multiple iterations. side_inputs = list(side_inputs) self.step_name = step_name self.context = DoFnContext(step_name, state=state) self.bundle_finalizer_param = DoFn.BundleFinalizerParam() do_fn_signature = DoFnSignature(fn) # Optimize for the common case. main_receivers = tagged_receivers[None] # TODO(BEAM-3937): Remove if block after output counter released. if 'outputs_per_element_counter' in RuntimeValueProvider.experiments: # TODO(BEAM-3955): Make step_name and operation_name less confused. output_counter_name = (CounterName('per-element-output-count', step_name=operation_name)) per_element_output_counter = state._counter_factory.get_counter( output_counter_name, Counter.DATAFLOW_DISTRIBUTION).accumulator else: per_element_output_counter = None output_processor = _OutputProcessor(windowing.windowfn, main_receivers, tagged_receivers, per_element_output_counter) if do_fn_signature.is_stateful_dofn() and not user_state_context: raise Exception( 'Requested execution of a stateful DoFn, but no user state context ' 'is available. This likely means that the current runner does not ' 'support the execution of stateful DoFns.') self.do_fn_invoker = DoFnInvoker.create_invoker( do_fn_signature, output_processor, self.context, side_inputs, args, kwargs, user_state_context=user_state_context, bundle_finalizer_param=self.bundle_finalizer_param)
def process(self, element, counter=DoFn.StateParam(BYTES_STATE)): return self.return_recursive(1)
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 onEndOfTime(self, batch=DoFn.StateParam(BATCH), batchSize=DoFn.StateParam(BATCH_SIZE)): return self.flush(batch, batchSize)