def to_runner_api(self, context): """Returns an SdkFunctionSpec encoding this Fn. Prefer overriding self.to_runner_api_parameter. """ from apache_beam.portability.api import beam_runner_api_pb2 urn, typed_param = self.to_runner_api_parameter(context) return beam_runner_api_pb2.SdkFunctionSpec( spec=beam_runner_api_pb2.FunctionSpec( urn=urn, any_param=proto_utils.pack_Any(typed_param), payload=typed_param.SerializeToString( ) if typed_param is not None else None))
def parse_coder(self, spec): context = pipeline_context.PipelineContext() coder_id = str(hash(str(spec))) component_ids = [ context.coders.get_id(self.parse_coder(c)) for c in spec.get('components', ()) ] context.coders.put_proto( coder_id, beam_runner_api_pb2.Coder(spec=beam_runner_api_pb2.FunctionSpec( urn=spec['urn'], payload=spec.get('payload')), component_coder_ids=component_ids)) return context.coders.get_by_id(coder_id)
def to_runner_api(self, context): """For internal use only; no backwards-compatibility guarantees. """ # TODO(BEAM-115): Use specialized URNs and components. serialized_coder = serialize_coder(self) return beam_runner_api_pb2.Coder( spec=beam_runner_api_pb2.SdkFunctionSpec( spec=beam_runner_api_pb2.FunctionSpec( urn=urns.PICKLED_CODER, any_param=proto_utils.pack_Any( google.protobuf.wrappers_pb2.BytesValue( value=serialized_coder)), payload=serialized_coder)))
def to_runner_api(self, context, has_parts=False, **extra_kwargs): # type: (PipelineContext, bool, Any) -> beam_runner_api_pb2.FunctionSpec from apache_beam.portability.api import beam_runner_api_pb2 urn, typed_param = self.to_runner_api_parameter( context, **extra_kwargs) if urn == python_urns.GENERIC_COMPOSITE_TRANSFORM and not has_parts: # TODO(BEAM-3812): Remove this fallback. urn, typed_param = self.to_runner_api_pickled(context) return beam_runner_api_pb2.FunctionSpec( urn=urn, payload=typed_param.SerializeToString() if isinstance( typed_param, message.Message) else typed_param.encode('utf-8') if isinstance(typed_param, str) else typed_param)
def to_runner_api(self, context, has_parts=False, **extra_kwargs): # type: (PipelineContext, bool, Any) -> beam_runner_api_pb2.FunctionSpec from apache_beam.portability.api import beam_runner_api_pb2 # typing: only ParDo supports extra_kwargs urn, typed_param = self.to_runner_api_parameter(context, **extra_kwargs) # type: ignore[call-arg] if urn == python_urns.GENERIC_COMPOSITE_TRANSFORM and not has_parts: # TODO(https://github.com/apache/beam/issues/18713): Remove this fallback. urn, typed_param = self.to_runner_api_pickled(context) return beam_runner_api_pb2.FunctionSpec( urn=urn, payload=typed_param.SerializeToString() if isinstance( typed_param, message.Message) else typed_param.encode('utf-8') if isinstance(typed_param, str) else typed_param)
def expand_gbk(stages): """Transforms each GBK into a write followed by a read. """ for stage in stages: assert len(stage.transforms) == 1 transform = stage.transforms[0] if transform.spec.urn == urns.GROUP_BY_KEY_ONLY_TRANSFORM: # This is used later to correlate the read and write. param = str("group:%s" % stage.name) gbk_write = Stage(transform.unique_name + '/Write', [ beam_runner_api_pb2.PTransform( unique_name=transform.unique_name + '/Write', inputs=transform.inputs, spec=beam_runner_api_pb2.FunctionSpec( urn=bundle_processor.DATA_OUTPUT_URN, any_param=proto_utils.pack_Any( wrappers_pb2.BytesValue(value=param)), payload=param)) ], downstream_side_inputs=frozenset(), must_follow=stage.must_follow) yield gbk_write yield Stage(transform.unique_name + '/Read', [ beam_runner_api_pb2.PTransform( unique_name=transform.unique_name + '/Read', outputs=transform.outputs, spec=beam_runner_api_pb2.FunctionSpec( urn=bundle_processor.DATA_INPUT_URN, any_param=proto_utils.pack_Any( wrappers_pb2.BytesValue(value=param)), payload=param)) ], downstream_side_inputs=frozenset(), must_follow=union(frozenset([gbk_write]), stage.must_follow)) else: yield stage
def impulse_to_input(stages, pipeline_context): """Translates Impulse operations into GRPC reads.""" for stage in stages: for transform in list(stage.transforms): if transform.spec.urn == common_urns.primitives.IMPULSE.urn: stage.transforms.remove(transform) stage.transforms.append( beam_runner_api_pb2.PTransform( unique_name=transform.unique_name, spec=beam_runner_api_pb2.FunctionSpec( urn=bundle_processor.DATA_INPUT_URN, payload=IMPULSE_BUFFER), outputs=transform.outputs)) yield stage
def wrap_unknown_coders(coder_id, with_bytes): if (coder_id, with_bytes) not in coder_substitutions: wrapped_coder_id = None coder_proto = coders[coder_id] if coder_proto.spec.spec.urn == urns.LENGTH_PREFIX_CODER: coder_substitutions[coder_id, with_bytes] = ( bytes_coder_id if with_bytes else coder_id) elif coder_proto.spec.spec.urn in good_coder_urns: wrapped_components = [ wrap_unknown_coders(c, with_bytes) for c in coder_proto.component_coder_ids ] if wrapped_components == list( coder_proto.component_coder_ids): # Use as is. coder_substitutions[coder_id, with_bytes] = coder_id else: wrapped_coder_id = unique_name( coders, coder_id + ("_bytes" if with_bytes else "_len_prefix")) coders[wrapped_coder_id].CopyFrom(coder_proto) coders[wrapped_coder_id].component_coder_ids[:] = [ wrap_unknown_coders(c, with_bytes) for c in coder_proto.component_coder_ids ] coder_substitutions[coder_id, with_bytes] = wrapped_coder_id else: # Not a known coder. if with_bytes: coder_substitutions[coder_id, with_bytes] = bytes_coder_id else: wrapped_coder_id = unique_name( coders, coder_id + "_len_prefix") len_prefix_coder_proto = beam_runner_api_pb2.Coder( spec=beam_runner_api_pb2.SdkFunctionSpec( spec=beam_runner_api_pb2.FunctionSpec( urn=urns.LENGTH_PREFIX_CODER)), component_coder_ids=[coder_id]) coders[wrapped_coder_id].CopyFrom( len_prefix_coder_proto) coder_substitutions[coder_id, with_bytes] = wrapped_coder_id # This operation is idempotent. if wrapped_coder_id: coder_substitutions[wrapped_coder_id, with_bytes] = wrapped_coder_id return coder_substitutions[coder_id, with_bytes]
def read_to_impulse(stages, pipeline_context): """Translates Read operations into Impulse operations.""" for stage in stages: # First map Reads, if any, to Impulse + triggered read op. for transform in list(stage.transforms): if transform.spec.urn == common_urns.deprecated_primitives.READ.urn: read_pc = only_element(transform.outputs.values()) read_pc_proto = pipeline_context.components.pcollections[ read_pc] impulse_pc = unique_name( pipeline_context.components.pcollections, 'Impulse') pipeline_context.components.pcollections[impulse_pc].CopyFrom( beam_runner_api_pb2.PCollection( unique_name=impulse_pc, coder_id=pipeline_context.bytes_coder_id, windowing_strategy_id=read_pc_proto. windowing_strategy_id, is_bounded=read_pc_proto.is_bounded)) stage.transforms.remove(transform) # TODO(robertwb): If this goes multi-process before fn-api # read is default, expand into split + reshuffle + read. stage.transforms.append( beam_runner_api_pb2.PTransform( unique_name=transform.unique_name + '/Impulse', spec=beam_runner_api_pb2.FunctionSpec( urn=common_urns.primitives.IMPULSE.urn), outputs={'out': impulse_pc})) stage.transforms.append( beam_runner_api_pb2.PTransform( unique_name=transform.unique_name, spec=beam_runner_api_pb2.FunctionSpec( urn=python_urns.IMPULSE_READ_TRANSFORM, payload=transform.spec.payload), inputs={'in': impulse_pc}, outputs={'out': read_pc})) yield stage
def to_runner_api(self, context): # type: (PipelineContext) -> beam_runner_api_pb2.SdkFunctionSpec """Returns an SdkFunctionSpec encoding this Fn. Prefer overriding self.to_runner_api_parameter. """ from apache_beam.portability.api import beam_runner_api_pb2 urn, typed_param = self.to_runner_api_parameter(context) return beam_runner_api_pb2.SdkFunctionSpec( environment_id=context.default_environment_id(), spec=beam_runner_api_pb2.FunctionSpec( urn=urn, payload=typed_param.SerializeToString() if isinstance(typed_param, message.Message) else typed_param))
def length_prefixed_and_safe_coder(self, coder_id): coder = self.components.coders[coder_id] if coder.spec.spec.urn == common_urns.coders.LENGTH_PREFIX.urn: return coder_id, self.bytes_coder_id elif coder.spec.spec.urn in self._KNOWN_CODER_URNS: new_component_ids = [ self.length_prefixed_coder(c) for c in coder.component_coder_ids ] if new_component_ids == coder.component_coder_ids: new_coder_id = coder_id else: new_coder_id = unique_name(self.components.coders, coder_id + '_length_prefixed') self.components.coders[new_coder_id].CopyFrom( beam_runner_api_pb2.Coder( spec=coder.spec, component_coder_ids=new_component_ids)) safe_component_ids = [ self.safe_coders[c] for c in new_component_ids ] if safe_component_ids == coder.component_coder_ids: safe_coder_id = coder_id else: safe_coder_id = unique_name(self.components.coders, coder_id + '_safe') self.components.coders[safe_coder_id].CopyFrom( beam_runner_api_pb2.Coder( spec=coder.spec, component_coder_ids=safe_component_ids)) return new_coder_id, safe_coder_id else: new_coder_id = unique_name(self.components.coders, coder_id + '_length_prefixed') self.components.coders[new_coder_id].CopyFrom( beam_runner_api_pb2.Coder( spec=beam_runner_api_pb2.SdkFunctionSpec( spec=beam_runner_api_pb2.FunctionSpec( urn=common_urns.coders.LENGTH_PREFIX.urn)), component_coder_ids=[coder_id])) return new_coder_id, self.bytes_coder_id
def fix_flatten_coders(stages, pipeline_context): """Ensures that the inputs of Flatten have the same coders as the output. """ pcollections = pipeline_context.components.pcollections for stage in stages: transform = only_element(stage.transforms) if transform.spec.urn == common_urns.primitives.FLATTEN.urn: output_pcoll_id = only_element(transform.outputs.values()) output_coder_id = pcollections[output_pcoll_id].coder_id for local_in, pcoll_in in list(transform.inputs.items()): if pcollections[pcoll_in].coder_id != output_coder_id: # Flatten requires that all its inputs be materialized with the # same coder as its output. Add stages to transcode flatten # inputs that use different coders. transcoded_pcollection = unique_name( pcollections, transform.unique_name + '/Transcode/' + local_in + '/out') transcode_name = unique_name( pipeline_context.components.transforms, transform.unique_name + '/Transcode/' + local_in) yield Stage( transcode_name, [beam_runner_api_pb2.PTransform( unique_name=transcode_name, inputs={local_in: pcoll_in}, outputs={'out': transcoded_pcollection}, spec=beam_runner_api_pb2.FunctionSpec( urn=bundle_processor.IDENTITY_DOFN_URN))], downstream_side_inputs=frozenset(), must_follow=stage.must_follow) pcollections[transcoded_pcollection].CopyFrom( pcollections[pcoll_in]) pcollections[transcoded_pcollection].unique_name = ( transcoded_pcollection) pcollections[transcoded_pcollection].coder_id = output_coder_id transform.inputs[local_in] = transcoded_pcollection yield stage
def greedily_fuse(stages): """Places transforms sharing an edge in the same stage, whenever possible. """ producers_by_pcoll = {} consumers_by_pcoll = collections.defaultdict(list) # Used to always reference the correct stage as the producer and # consumer maps are not updated when stages are fused away. replacements = {} def replacement(s): old_ss = [] while s in replacements: old_ss.append(s) s = replacements[s] for old_s in old_ss[:-1]: replacements[old_s] = s return s def fuse(producer, consumer): fused = producer.fuse(consumer) replacements[producer] = fused replacements[consumer] = fused # First record the producers and consumers of each PCollection. for stage in stages: for transform in stage.transforms: for input in transform.inputs.values(): consumers_by_pcoll[input].append(stage) for output in transform.outputs.values(): producers_by_pcoll[output] = stage logging.debug('consumers\n%s', consumers_by_pcoll) logging.debug('producers\n%s', producers_by_pcoll) # Now try to fuse away all pcollections. for pcoll, producer in producers_by_pcoll.items(): pcoll_as_param = str("materialize:%s" % pcoll) write_pcoll = None for consumer in consumers_by_pcoll[pcoll]: producer = replacement(producer) consumer = replacement(consumer) # Update consumer.must_follow set, as it's used in can_fuse. consumer.must_follow = frozenset( replacement(s) for s in consumer.must_follow) if producer.can_fuse(consumer): fuse(producer, consumer) else: # If we can't fuse, do a read + write. if write_pcoll is None: write_pcoll = Stage(pcoll + '/Write', [ beam_runner_api_pb2.PTransform( unique_name=pcoll + '/Write', inputs={'in': pcoll}, spec=beam_runner_api_pb2.FunctionSpec( urn=bundle_processor.DATA_OUTPUT_URN, payload=pcoll_as_param)) ]) fuse(producer, write_pcoll) if consumer.has_as_main_input(pcoll): read_pcoll = Stage(pcoll + '/Read', [ beam_runner_api_pb2.PTransform( unique_name=pcoll + '/Read', outputs={'out': pcoll}, spec=beam_runner_api_pb2.FunctionSpec( urn=bundle_processor.DATA_INPUT_URN, payload=pcoll_as_param)) ], must_follow=frozenset( [write_pcoll])) fuse(read_pcoll, consumer) else: consumer.must_follow = union( consumer.must_follow, frozenset([write_pcoll])) # Everything that was originally a stage or a replacement, but wasn't # replaced, should be in the final graph. final_stages = frozenset(stages).union( replacements.values()).difference(replacements.keys()) for stage in final_stages: # Update all references to their final values before throwing # the replacement data away. stage.must_follow = frozenset( replacement(s) for s in stage.must_follow) # Two reads of the same stage may have been fused. This is unneeded. stage.deduplicate_read() return final_stages
def to_runner_api(self, context): from apache_beam.portability.api import beam_runner_api_pb2 urn, typed_param = self.to_runner_api_parameter(context) return beam_runner_api_pb2.FunctionSpec( urn=urn, parameter=proto_utils.pack_Any(typed_param))
def sink_flattens(stages): """Sink flattens and remove them from the graph. A flatten that cannot be sunk/fused away becomes multiple writes (to the same logical sink) followed by a read. """ # TODO(robertwb): Actually attempt to sink rather than always materialize. # TODO(robertwb): Possibly fuse this into one of the stages. pcollections = pipeline_components.pcollections for stage in stages: assert len(stage.transforms) == 1 transform = stage.transforms[0] if transform.spec.urn == urns.FLATTEN_TRANSFORM: # This is used later to correlate the read and writes. param = str("materialize:%s" % transform.unique_name) output_pcoll_id, = transform.outputs.values() output_coder_id = pcollections[output_pcoll_id].coder_id flatten_writes = [] for local_in, pcoll_in in transform.inputs.items(): if pcollections[pcoll_in].coder_id != output_coder_id: # Flatten inputs must all be written with the same coder as is # used to read them. pcollections[pcoll_in].coder_id = output_coder_id transcoded_pcollection = (transform.unique_name + '/Transcode/' + local_in + '/out') yield Stage( transform.unique_name + '/Transcode/' + local_in, [ beam_runner_api_pb2.PTransform( unique_name=transform.unique_name + '/Transcode/' + local_in, inputs={local_in: pcoll_in}, outputs={ 'out': transcoded_pcollection }, spec=beam_runner_api_pb2.FunctionSpec( urn=bundle_processor. IDENTITY_DOFN_URN)) ], downstream_side_inputs=frozenset(), must_follow=stage.must_follow) pcollections[transcoded_pcollection].CopyFrom( pcollections[pcoll_in]) pcollections[ transcoded_pcollection].coder_id = output_coder_id else: transcoded_pcollection = pcoll_in flatten_write = Stage( transform.unique_name + '/Write/' + local_in, [ beam_runner_api_pb2.PTransform( unique_name=transform.unique_name + '/Write/' + local_in, inputs={local_in: transcoded_pcollection}, spec=beam_runner_api_pb2.FunctionSpec( urn=bundle_processor.DATA_OUTPUT_URN, payload=param)) ], downstream_side_inputs=frozenset(), must_follow=stage.must_follow) flatten_writes.append(flatten_write) yield flatten_write yield Stage(transform.unique_name + '/Read', [ beam_runner_api_pb2.PTransform( unique_name=transform.unique_name + '/Read', outputs=transform.outputs, spec=beam_runner_api_pb2.FunctionSpec( urn=bundle_processor.DATA_INPUT_URN, payload=param)) ], downstream_side_inputs=frozenset(), must_follow=union(frozenset(flatten_writes), stage.must_follow)) else: yield stage
def _map_task_to_protos(self, map_task, data_operation_spec): input_data = {} side_input_data = {} runner_sinks = {} context = pipeline_context.PipelineContext() transform_protos = {} used_pcollections = {} def uniquify(*names): # An injective mapping from string* to string. return ':'.join("%s:%d" % (name, len(name)) for name in names) def pcollection_id(op_ix, out_ix): if (op_ix, out_ix) not in used_pcollections: used_pcollections[op_ix, out_ix] = uniquify( map_task[op_ix][0], 'out', str(out_ix)) return used_pcollections[op_ix, out_ix] def get_inputs(op): if hasattr(op, 'inputs'): inputs = op.inputs elif hasattr(op, 'input'): inputs = [op.input] else: inputs = [] return {'in%s' % ix: pcollection_id(*input) for ix, input in enumerate(inputs)} def get_outputs(op_ix): op = map_task[op_ix][1] return {tag: pcollection_id(op_ix, out_ix) for out_ix, tag in enumerate(getattr(op, 'output_tags', ['out']))} for op_ix, (stage_name, operation) in enumerate(map_task): transform_id = uniquify(stage_name) if isinstance(operation, operation_specs.WorkerInMemoryWrite): # Write this data back to the runner. target_name = only_element(get_inputs(operation).keys()) runner_sinks[(transform_id, target_name)] = operation transform_spec = beam_runner_api_pb2.FunctionSpec( urn=bundle_processor.DATA_OUTPUT_URN, any_param=proto_utils.pack_Any(data_operation_spec), payload=data_operation_spec.SerializeToString() \ if data_operation_spec is not None else None) elif isinstance(operation, operation_specs.WorkerRead): # A Read from an in-memory source is done over the data plane. if (isinstance(operation.source.source, maptask_executor_runner.InMemorySource) and isinstance(operation.source.source.default_output_coder(), WindowedValueCoder)): target_name = only_element(get_outputs(op_ix).keys()) input_data[(transform_id, target_name)] = self._reencode_elements( operation.source.source.read(None), operation.source.source.default_output_coder()) transform_spec = beam_runner_api_pb2.FunctionSpec( urn=bundle_processor.DATA_INPUT_URN, any_param=proto_utils.pack_Any(data_operation_spec), payload=data_operation_spec.SerializeToString() \ if data_operation_spec is not None else None) else: # Otherwise serialize the source and execute it there. # TODO: Use SDFs with an initial impulse. # The Dataflow runner harness strips the base64 encoding. do the same # here until we get the same thing back that we sent in. source_bytes = base64.b64decode( pickler.dumps(operation.source.source)) transform_spec = beam_runner_api_pb2.FunctionSpec( urn=bundle_processor.PYTHON_SOURCE_URN, any_param=proto_utils.pack_Any( wrappers_pb2.BytesValue( value=source_bytes)), payload=source_bytes) elif isinstance(operation, operation_specs.WorkerDoFn): # Record the contents of each side input for access via the state api. side_input_extras = [] for si in operation.side_inputs: assert isinstance(si.source, iobase.BoundedSource) element_coder = si.source.default_output_coder() # TODO(robertwb): Actually flesh out the ViewFn API. side_input_extras.append((si.tag, element_coder)) side_input_data[ bundle_processor.side_input_tag(transform_id, si.tag)] = ( self._reencode_elements( si.source.read(si.source.get_range_tracker(None, None)), element_coder)) augmented_serialized_fn = pickler.dumps( (operation.serialized_fn, side_input_extras)) transform_spec = beam_runner_api_pb2.FunctionSpec( urn=bundle_processor.PYTHON_DOFN_URN, any_param=proto_utils.pack_Any( wrappers_pb2.BytesValue(value=augmented_serialized_fn)), payload=augmented_serialized_fn) elif isinstance(operation, operation_specs.WorkerFlatten): # Flatten is nice and simple. transform_spec = beam_runner_api_pb2.FunctionSpec( urn=bundle_processor.IDENTITY_DOFN_URN) else: raise NotImplementedError(operation) transform_protos[transform_id] = beam_runner_api_pb2.PTransform( unique_name=stage_name, spec=transform_spec, inputs=get_inputs(operation), outputs=get_outputs(op_ix)) pcollection_protos = { name: beam_runner_api_pb2.PCollection( unique_name=name, coder_id=context.coders.get_id( map_task[op_id][1].output_coders[out_id])) for (op_id, out_id), name in used_pcollections.items() } # Must follow creation of pcollection_protos to capture used coders. context_proto = context.to_runner_api() process_bundle_descriptor = beam_fn_api_pb2.ProcessBundleDescriptor( id=self._next_uid(), transforms=transform_protos, pcollections=pcollection_protos, coders=dict(context_proto.coders.items()), windowing_strategies=dict(context_proto.windowing_strategies.items()), environments=dict(context_proto.environments.items())) return input_data, side_input_data, runner_sinks, process_bundle_descriptor
def expand_gbk(stages): """Transforms each GBK into a write followed by a read. """ good_coder_urns = set(beam.coders.Coder._known_urns.keys()) - set( [urns.PICKLED_CODER]) coders = pipeline_components.coders for coder_id, coder_proto in coders.items(): if coder_proto.spec.spec.urn == urns.BYTES_CODER: bytes_coder_id = coder_id break else: bytes_coder_id = unique_name(coders, 'bytes_coder') pipeline_components.coders[bytes_coder_id].CopyFrom( beam.coders.BytesCoder().to_runner_api(None)) coder_substitutions = {} def wrap_unknown_coders(coder_id, with_bytes): if (coder_id, with_bytes) not in coder_substitutions: wrapped_coder_id = None coder_proto = coders[coder_id] if coder_proto.spec.spec.urn == urns.LENGTH_PREFIX_CODER: coder_substitutions[coder_id, with_bytes] = ( bytes_coder_id if with_bytes else coder_id) elif coder_proto.spec.spec.urn in good_coder_urns: wrapped_components = [ wrap_unknown_coders(c, with_bytes) for c in coder_proto.component_coder_ids ] if wrapped_components == list( coder_proto.component_coder_ids): # Use as is. coder_substitutions[coder_id, with_bytes] = coder_id else: wrapped_coder_id = unique_name( coders, coder_id + ("_bytes" if with_bytes else "_len_prefix")) coders[wrapped_coder_id].CopyFrom(coder_proto) coders[wrapped_coder_id].component_coder_ids[:] = [ wrap_unknown_coders(c, with_bytes) for c in coder_proto.component_coder_ids ] coder_substitutions[coder_id, with_bytes] = wrapped_coder_id else: # Not a known coder. if with_bytes: coder_substitutions[coder_id, with_bytes] = bytes_coder_id else: wrapped_coder_id = unique_name( coders, coder_id + "_len_prefix") len_prefix_coder_proto = beam_runner_api_pb2.Coder( spec=beam_runner_api_pb2.SdkFunctionSpec( spec=beam_runner_api_pb2.FunctionSpec( urn=urns.LENGTH_PREFIX_CODER)), component_coder_ids=[coder_id]) coders[wrapped_coder_id].CopyFrom( len_prefix_coder_proto) coder_substitutions[coder_id, with_bytes] = wrapped_coder_id # This operation is idempotent. if wrapped_coder_id: coder_substitutions[wrapped_coder_id, with_bytes] = wrapped_coder_id return coder_substitutions[coder_id, with_bytes] def fix_pcoll_coder(pcoll): new_coder_id = wrap_unknown_coders(pcoll.coder_id, False) safe_coders[new_coder_id] = wrap_unknown_coders( pcoll.coder_id, True) pcoll.coder_id = new_coder_id for stage in stages: assert len(stage.transforms) == 1 transform = stage.transforms[0] if transform.spec.urn == urns.GROUP_BY_KEY_TRANSFORM: for pcoll_id in transform.inputs.values(): fix_pcoll_coder( pipeline_components.pcollections[pcoll_id]) for pcoll_id in transform.outputs.values(): fix_pcoll_coder( pipeline_components.pcollections[pcoll_id]) # This is used later to correlate the read and write. param = str("group:%s" % stage.name) gbk_write = Stage(transform.unique_name + '/Write', [ beam_runner_api_pb2.PTransform( unique_name=transform.unique_name + '/Write', inputs=transform.inputs, spec=beam_runner_api_pb2.FunctionSpec( urn=bundle_processor.DATA_OUTPUT_URN, payload=param)) ], downstream_side_inputs=frozenset(), must_follow=stage.must_follow) yield gbk_write yield Stage(transform.unique_name + '/Read', [ beam_runner_api_pb2.PTransform( unique_name=transform.unique_name + '/Read', outputs=transform.outputs, spec=beam_runner_api_pb2.FunctionSpec( urn=bundle_processor.DATA_INPUT_URN, payload=param)) ], downstream_side_inputs=frozenset(), must_follow=union(frozenset([gbk_write]), stage.must_follow)) else: yield stage
def expand(self, pvalueish): # type: (pvalue.PCollection) -> pvalue.PCollection if isinstance(pvalueish, pvalue.PBegin): self._inputs = {} elif isinstance(pvalueish, (list, tuple)): self._inputs = { str(ix): pvalue for ix, pvalue in enumerate(pvalueish) } elif isinstance(pvalueish, dict): self._inputs = pvalueish else: self._inputs = {'input': pvalueish} pipeline = (next(iter(self._inputs.values())).pipeline if self._inputs else pvalueish.pipeline) context = pipeline_context.PipelineContext( component_id_map=pipeline.component_id_map) transform_proto = beam_runner_api_pb2.PTransform( unique_name=pipeline._current_transform().full_label, spec=beam_runner_api_pb2.FunctionSpec(urn=self._urn, payload=self._payload)) for tag, pcoll in self._inputs.items(): transform_proto.inputs[tag] = context.pcollections.get_id(pcoll) # Conversion to/from proto assumes producers. # TODO: Possibly loosen this. context.transforms.put_proto( '%s_%s' % (self._IMPULSE_PREFIX, tag), beam_runner_api_pb2.PTransform( unique_name='%s_%s' % (self._IMPULSE_PREFIX, tag), spec=beam_runner_api_pb2.FunctionSpec( urn=common_urns.primitives.IMPULSE.urn), outputs={'out': transform_proto.inputs[tag]})) components = context.to_runner_api() request = beam_expansion_api_pb2.ExpansionRequest( components=components, namespace=self. _external_namespace, # type: ignore # mypy thinks self._namespace is threading.local transform=transform_proto) with self._service() as service: response = service.Expand(request) if response.error: raise RuntimeError(response.error) self._expanded_components = response.components if any(env.dependencies for env in self._expanded_components.environments.values()): self._expanded_components = self._resolve_artifacts( self._expanded_components, service.artifact_service(), pipeline.local_tempdir) self._expanded_transform = response.transform self._expanded_requirements = response.requirements result_context = pipeline_context.PipelineContext(response.components) def fix_output(pcoll, tag): pcoll.pipeline = pipeline pcoll.tag = tag return pcoll self._outputs = { tag: fix_output(result_context.pcollections.get_by_id(pcoll_id), tag) for tag, pcoll_id in self._expanded_transform.outputs.items() } return self._output_to_pvalueish(self._outputs)
def expand(self, pvalueish): if isinstance(pvalueish, pvalue.PBegin): self._inputs = {} elif isinstance(pvalueish, (list, tuple)): self._inputs = { str(ix): pvalue for ix, pvalue in enumerate(pvalueish) } elif isinstance(pvalueish, dict): self._inputs = pvalueish else: self._inputs = {'input': pvalueish} pipeline = (next(iter(self._inputs.values())).pipeline if self._inputs else pvalueish.pipeline) context = pipeline_context.PipelineContext() transform_proto = beam_runner_api_pb2.PTransform( unique_name=self._EXPANDED_TRANSFORM_UNIQUE_NAME, spec=beam_runner_api_pb2.FunctionSpec(urn=self._urn, payload=self._payload)) for tag, pcoll in self._inputs.items(): transform_proto.inputs[tag] = context.pcollections.get_id(pcoll) # Conversion to/from proto assumes producers. # TODO: Possibly loosen this. context.transforms.put_proto( '%s_%s' % (self._IMPULSE_PREFIX, tag), beam_runner_api_pb2.PTransform( unique_name='%s_%s' % (self._IMPULSE_PREFIX, tag), spec=beam_runner_api_pb2.FunctionSpec( urn=common_urns.primitives.IMPULSE.urn), outputs={'out': transform_proto.inputs[tag]})) components = context.to_runner_api() request = beam_expansion_api_pb2.ExpansionRequest( components=components, namespace=self._namespace, transform=transform_proto) if isinstance(self._endpoint, str): with grpc.insecure_channel(self._endpoint) as channel: response = beam_expansion_api_pb2_grpc.ExpansionServiceStub( channel).Expand(request) else: response = self._endpoint.Expand(request, None) if response.error: raise RuntimeError(response.error) self._expanded_components = response.components self._expanded_transform = response.transform result_context = pipeline_context.PipelineContext(response.components) def fix_output(pcoll, tag): pcoll.pipeline = pipeline pcoll.tag = tag return pcoll self._outputs = { tag: fix_output(result_context.pcollections.get_by_id(pcoll_id), tag) for tag, pcoll_id in self._expanded_transform.outputs.items() } return self._output_to_pvalueish(self._outputs)
def lift_combiners(stages): """Expands CombinePerKey into pre- and post-grouping stages. ... -> CombinePerKey -> ... becomes ... -> PreCombine -> GBK -> MergeAccumulators -> ExtractOutput -> ... """ def add_or_get_coder_id(coder_proto): for coder_id, coder in pipeline_components.coders.items(): if coder == coder_proto: return coder_id new_coder_id = unique_name(pipeline_components.coders, 'coder') pipeline_components.coders[new_coder_id].CopyFrom(coder_proto) return new_coder_id def windowed_coder_id(coder_id): proto = beam_runner_api_pb2.Coder( spec=beam_runner_api_pb2.SdkFunctionSpec( spec=beam_runner_api_pb2.FunctionSpec( urn=urns.WINDOWED_VALUE_CODER)), component_coder_ids=[coder_id, window_coder_id]) return add_or_get_coder_id(proto) for stage in stages: assert len(stage.transforms) == 1 transform = stage.transforms[0] if transform.spec.urn == urns.COMBINE_PER_KEY_TRANSFORM: combine_payload = proto_utils.parse_Bytes( transform.spec.payload, beam_runner_api_pb2.CombinePayload) input_pcoll = pipeline_components.pcollections[only_element( transform.inputs.values())] output_pcoll = pipeline_components.pcollections[only_element( transform.outputs.values())] windowed_input_coder = pipeline_components.coders[ input_pcoll.coder_id] element_coder_id, window_coder_id = ( windowed_input_coder.component_coder_ids) element_coder = pipeline_components.coders[element_coder_id] key_coder_id, _ = element_coder.component_coder_ids accumulator_coder_id = combine_payload.accumulator_coder_id key_accumulator_coder = beam_runner_api_pb2.Coder( spec=beam_runner_api_pb2.SdkFunctionSpec( spec=beam_runner_api_pb2.FunctionSpec( urn=urns.KV_CODER)), component_coder_ids=[key_coder_id, accumulator_coder_id]) key_accumulator_coder_id = add_or_get_coder_id(key_accumulator_coder) accumulator_iter_coder = beam_runner_api_pb2.Coder( spec=beam_runner_api_pb2.SdkFunctionSpec( spec=beam_runner_api_pb2.FunctionSpec( urn=urns.ITERABLE_CODER)), component_coder_ids=[accumulator_coder_id]) accumulator_iter_coder_id = add_or_get_coder_id( accumulator_iter_coder) key_accumulator_iter_coder = beam_runner_api_pb2.Coder( spec=beam_runner_api_pb2.SdkFunctionSpec( spec=beam_runner_api_pb2.FunctionSpec( urn=urns.KV_CODER)), component_coder_ids=[key_coder_id, accumulator_iter_coder_id]) key_accumulator_iter_coder_id = add_or_get_coder_id( key_accumulator_iter_coder) precombined_pcoll_id = unique_name( pipeline_components.pcollections, 'pcollection') pipeline_components.pcollections[precombined_pcoll_id].CopyFrom( beam_runner_api_pb2.PCollection( unique_name=transform.unique_name + '/Precombine.out', coder_id=windowed_coder_id(key_accumulator_coder_id), windowing_strategy_id=input_pcoll.windowing_strategy_id, is_bounded=input_pcoll.is_bounded)) grouped_pcoll_id = unique_name( pipeline_components.pcollections, 'pcollection') pipeline_components.pcollections[grouped_pcoll_id].CopyFrom( beam_runner_api_pb2.PCollection( unique_name=transform.unique_name + '/Group.out', coder_id=windowed_coder_id(key_accumulator_iter_coder_id), windowing_strategy_id=output_pcoll.windowing_strategy_id, is_bounded=output_pcoll.is_bounded)) merged_pcoll_id = unique_name( pipeline_components.pcollections, 'pcollection') pipeline_components.pcollections[merged_pcoll_id].CopyFrom( beam_runner_api_pb2.PCollection( unique_name=transform.unique_name + '/Merge.out', coder_id=windowed_coder_id(key_accumulator_coder_id), windowing_strategy_id=output_pcoll.windowing_strategy_id, is_bounded=output_pcoll.is_bounded)) def make_stage(base_stage, transform): return Stage( transform.unique_name, [transform], downstream_side_inputs=base_stage.downstream_side_inputs, must_follow=base_stage.must_follow) yield make_stage( stage, beam_runner_api_pb2.PTransform( unique_name=transform.unique_name + '/Precombine', spec=beam_runner_api_pb2.FunctionSpec( urn=urns.PRECOMBINE_TRANSFORM, payload=transform.spec.payload), inputs=transform.inputs, outputs={'out': precombined_pcoll_id})) yield make_stage( stage, beam_runner_api_pb2.PTransform( unique_name=transform.unique_name + '/Group', spec=beam_runner_api_pb2.FunctionSpec( urn=urns.GROUP_BY_KEY_TRANSFORM), inputs={'in': precombined_pcoll_id}, outputs={'out': grouped_pcoll_id})) yield make_stage( stage, beam_runner_api_pb2.PTransform( unique_name=transform.unique_name + '/Merge', spec=beam_runner_api_pb2.FunctionSpec( urn=urns.MERGE_ACCUMULATORS_TRANSFORM, payload=transform.spec.payload), inputs={'in': grouped_pcoll_id}, outputs={'out': merged_pcoll_id})) yield make_stage( stage, beam_runner_api_pb2.PTransform( unique_name=transform.unique_name + '/ExtractOutputs', spec=beam_runner_api_pb2.FunctionSpec( urn=urns.EXTRACT_OUTPUTS_TRANSFORM, payload=transform.spec.payload), inputs={'in': merged_pcoll_id}, outputs=transform.outputs)) else: yield stage
def inject_timer_pcollections(stages, pipeline_context): """Create PCollections for fired timers and to-be-set timers. At execution time, fired timers and timers-to-set are represented as PCollections that are managed by the runner. This phase adds the necissary collections, with their read and writes, to any stages using timers. """ for stage in stages: for transform in list(stage.transforms): if transform.spec.urn == common_urns.primitives.PAR_DO.urn: payload = proto_utils.parse_Bytes( transform.spec.payload, beam_runner_api_pb2.ParDoPayload) for tag, spec in payload.timer_specs.items(): if len(transform.inputs) > 1: raise NotImplementedError('Timers and side inputs.') input_pcoll = pipeline_context.components.pcollections[ next(iter(transform.inputs.values()))] # Create the appropriate coder for the timer PCollection. key_coder_id = input_pcoll.coder_id if (pipeline_context.components.coders[key_coder_id].spec. spec.urn == common_urns.coders.KV.urn): key_coder_id = pipeline_context.components.coders[ key_coder_id].component_coder_ids[0] key_timer_coder_id = pipeline_context.add_or_get_coder_id( beam_runner_api_pb2.Coder( spec=beam_runner_api_pb2.SdkFunctionSpec( spec=beam_runner_api_pb2.FunctionSpec( urn=common_urns.coders.KV.urn)), component_coder_ids=[ key_coder_id, spec.timer_coder_id ])) # Inject the read and write pcollections. timer_read_pcoll = unique_name( pipeline_context.components.pcollections, '%s_timers_to_read_%s' % (transform.unique_name, tag)) timer_write_pcoll = unique_name( pipeline_context.components.pcollections, '%s_timers_to_write_%s' % (transform.unique_name, tag)) pipeline_context.components.pcollections[ timer_read_pcoll].CopyFrom( beam_runner_api_pb2.PCollection( unique_name=timer_read_pcoll, coder_id=key_timer_coder_id, windowing_strategy_id=input_pcoll. windowing_strategy_id, is_bounded=input_pcoll.is_bounded)) pipeline_context.components.pcollections[ timer_write_pcoll].CopyFrom( beam_runner_api_pb2.PCollection( unique_name=timer_write_pcoll, coder_id=key_timer_coder_id, windowing_strategy_id=input_pcoll. windowing_strategy_id, is_bounded=input_pcoll.is_bounded)) stage.transforms.append( beam_runner_api_pb2.PTransform( unique_name=timer_read_pcoll + '/Read', outputs={'out': timer_read_pcoll}, spec=beam_runner_api_pb2.FunctionSpec( urn=bundle_processor.DATA_INPUT_URN, payload=create_buffer_id(timer_read_pcoll, kind='timers')))) stage.transforms.append( beam_runner_api_pb2.PTransform( unique_name=timer_write_pcoll + '/Write', inputs={'in': timer_write_pcoll}, spec=beam_runner_api_pb2.FunctionSpec( urn=bundle_processor.DATA_OUTPUT_URN, payload=create_buffer_id(timer_write_pcoll, kind='timers')))) assert tag not in transform.inputs transform.inputs[tag] = timer_read_pcoll assert tag not in transform.outputs transform.outputs[tag] = timer_write_pcoll stage.timer_pcollections.append( (timer_read_pcoll + '/Read', timer_write_pcoll)) yield stage
def expand(self, pvalueish): # type: (pvalue.PCollection) -> pvalue.PCollection if isinstance(pvalueish, pvalue.PBegin): self._inputs = {} elif isinstance(pvalueish, (list, tuple)): self._inputs = { str(ix): pvalue for ix, pvalue in enumerate(pvalueish) } elif isinstance(pvalueish, dict): self._inputs = pvalueish else: self._inputs = {'input': pvalueish} pipeline = (next(iter(self._inputs.values())).pipeline if self._inputs else pvalueish.pipeline) context = pipeline_context.PipelineContext() transform_proto = beam_runner_api_pb2.PTransform( unique_name=pipeline._current_transform().full_label, spec=beam_runner_api_pb2.FunctionSpec(urn=self._urn, payload=self._payload)) for tag, pcoll in self._inputs.items(): transform_proto.inputs[tag] = context.pcollections.get_id(pcoll) # Conversion to/from proto assumes producers. # TODO: Possibly loosen this. context.transforms.put_proto( '%s_%s' % (self._IMPULSE_PREFIX, tag), beam_runner_api_pb2.PTransform( unique_name='%s_%s' % (self._IMPULSE_PREFIX, tag), spec=beam_runner_api_pb2.FunctionSpec( urn=common_urns.primitives.IMPULSE.urn), outputs={'out': transform_proto.inputs[tag]})) components = context.to_runner_api() request = beam_expansion_api_pb2.ExpansionRequest( components=components, namespace=self. _namespace, # type: ignore # mypy thinks self._namespace is threading.local transform=transform_proto) if isinstance(self._expansion_service, str): # Some environments may not support unsecure channels. Hence using a # secure channel with local credentials here. # TODO: update this to support secure non-local channels. channel_creds = grpc.local_channel_credentials() with grpc.secure_channel(self._expansion_service, channel_creds) as channel: response = beam_expansion_api_pb2_grpc.ExpansionServiceStub( channel).Expand(request) else: response = self._expansion_service.Expand(request, None) if response.error: raise RuntimeError(response.error) self._expanded_components = response.components self._expanded_transform = response.transform self._expanded_requirements = response.requirements result_context = pipeline_context.PipelineContext(response.components) def fix_output(pcoll, tag): pcoll.pipeline = pipeline pcoll.tag = tag return pcoll self._outputs = { tag: fix_output(result_context.pcollections.get_by_id(pcoll_id), tag) for tag, pcoll_id in self._expanded_transform.outputs.items() } return self._output_to_pvalueish(self._outputs)
def lift_combiners(stages, context): """Expands CombinePerKey into pre- and post-grouping stages. ... -> CombinePerKey -> ... becomes ... -> PreCombine -> GBK -> MergeAccumulators -> ExtractOutput -> ... """ for stage in stages: assert len(stage.transforms) == 1 transform = stage.transforms[0] if transform.spec.urn == common_urns.composites.COMBINE_PER_KEY.urn: combine_payload = proto_utils.parse_Bytes( transform.spec.payload, beam_runner_api_pb2.CombinePayload) input_pcoll = context.components.pcollections[only_element( list(transform.inputs.values()))] output_pcoll = context.components.pcollections[only_element( list(transform.outputs.values()))] element_coder_id = input_pcoll.coder_id element_coder = context.components.coders[element_coder_id] key_coder_id, _ = element_coder.component_coder_ids accumulator_coder_id = combine_payload.accumulator_coder_id key_accumulator_coder = beam_runner_api_pb2.Coder( spec=beam_runner_api_pb2.SdkFunctionSpec( spec=beam_runner_api_pb2.FunctionSpec( urn=common_urns.coders.KV.urn)), component_coder_ids=[key_coder_id, accumulator_coder_id]) key_accumulator_coder_id = context.add_or_get_coder_id( key_accumulator_coder) accumulator_iter_coder = beam_runner_api_pb2.Coder( spec=beam_runner_api_pb2.SdkFunctionSpec( spec=beam_runner_api_pb2.FunctionSpec( urn=common_urns.coders.ITERABLE.urn)), component_coder_ids=[accumulator_coder_id]) accumulator_iter_coder_id = context.add_or_get_coder_id( accumulator_iter_coder) key_accumulator_iter_coder = beam_runner_api_pb2.Coder( spec=beam_runner_api_pb2.SdkFunctionSpec( spec=beam_runner_api_pb2.FunctionSpec( urn=common_urns.coders.KV.urn)), component_coder_ids=[key_coder_id, accumulator_iter_coder_id]) key_accumulator_iter_coder_id = context.add_or_get_coder_id( key_accumulator_iter_coder) precombined_pcoll_id = unique_name(context.components.pcollections, 'pcollection') context.components.pcollections[precombined_pcoll_id].CopyFrom( beam_runner_api_pb2.PCollection( unique_name=transform.unique_name + '/Precombine.out', coder_id=key_accumulator_coder_id, windowing_strategy_id=input_pcoll.windowing_strategy_id, is_bounded=input_pcoll.is_bounded)) grouped_pcoll_id = unique_name(context.components.pcollections, 'pcollection') context.components.pcollections[grouped_pcoll_id].CopyFrom( beam_runner_api_pb2.PCollection( unique_name=transform.unique_name + '/Group.out', coder_id=key_accumulator_iter_coder_id, windowing_strategy_id=output_pcoll.windowing_strategy_id, is_bounded=output_pcoll.is_bounded)) merged_pcoll_id = unique_name(context.components.pcollections, 'pcollection') context.components.pcollections[merged_pcoll_id].CopyFrom( beam_runner_api_pb2.PCollection( unique_name=transform.unique_name + '/Merge.out', coder_id=key_accumulator_coder_id, windowing_strategy_id=output_pcoll.windowing_strategy_id, is_bounded=output_pcoll.is_bounded)) def make_stage(base_stage, transform): return Stage( transform.unique_name, [transform], downstream_side_inputs=base_stage.downstream_side_inputs, must_follow=base_stage.must_follow, parent=base_stage.name) yield make_stage( stage, beam_runner_api_pb2.PTransform( unique_name=transform.unique_name + '/Precombine', spec=beam_runner_api_pb2.FunctionSpec( urn=common_urns.combine_components. COMBINE_PER_KEY_PRECOMBINE.urn, payload=transform.spec.payload), inputs=transform.inputs, outputs={'out': precombined_pcoll_id})) yield make_stage( stage, beam_runner_api_pb2.PTransform( unique_name=transform.unique_name + '/Group', spec=beam_runner_api_pb2.FunctionSpec( urn=common_urns.primitives.GROUP_BY_KEY.urn), inputs={'in': precombined_pcoll_id}, outputs={'out': grouped_pcoll_id})) yield make_stage( stage, beam_runner_api_pb2.PTransform( unique_name=transform.unique_name + '/Merge', spec=beam_runner_api_pb2.FunctionSpec( urn=common_urns.combine_components. COMBINE_PER_KEY_MERGE_ACCUMULATORS.urn, payload=transform.spec.payload), inputs={'in': grouped_pcoll_id}, outputs={'out': merged_pcoll_id})) yield make_stage( stage, beam_runner_api_pb2.PTransform( unique_name=transform.unique_name + '/ExtractOutputs', spec=beam_runner_api_pb2.FunctionSpec( urn=common_urns.combine_components. COMBINE_PER_KEY_EXTRACT_OUTPUTS.urn, payload=transform.spec.payload), inputs={'in': merged_pcoll_id}, outputs=transform.outputs)) else: yield stage
def make_process_bundle_descriptor(self, data_api_service_descriptor, state_api_service_descriptor): # type: (Optional[endpoints_pb2.ApiServiceDescriptor], Optional[endpoints_pb2.ApiServiceDescriptor]) -> beam_fn_api_pb2.ProcessBundleDescriptor """Creates a ProcessBundleDescriptor for invoking the WindowFn's merge operation. """ def make_channel_payload(coder_id): # type: (str) -> bytes data_spec = beam_fn_api_pb2.RemoteGrpcPort(coder_id=coder_id) if data_api_service_descriptor: data_spec.api_service_descriptor.url = ( data_api_service_descriptor.url) return data_spec.SerializeToString() pipeline_context = self._execution_context_ref().pipeline_context global_windowing_strategy_id = self.uid('global_windowing_strategy') global_windowing_strategy_proto = core.Windowing( window.GlobalWindows()).to_runner_api(pipeline_context) coders = dict(pipeline_context.coders.get_id_to_proto_map()) def make_coder(urn, *components): # type: (str, str) -> str coder_proto = beam_runner_api_pb2.Coder( spec=beam_runner_api_pb2.FunctionSpec(urn=urn), component_coder_ids=components) coder_id = self.uid('coder') coders[coder_id] = coder_proto pipeline_context.coders.put_proto(coder_id, coder_proto) return coder_id bytes_coder_id = make_coder(common_urns.coders.BYTES.urn) window_coder_id = self._windowing_strategy_proto.window_coder_id global_window_coder_id = make_coder( common_urns.coders.GLOBAL_WINDOW.urn) iter_window_coder_id = make_coder(common_urns.coders.ITERABLE.urn, window_coder_id) input_coder_id = make_coder(common_urns.coders.KV.urn, bytes_coder_id, iter_window_coder_id) output_coder_id = make_coder( common_urns.coders.KV.urn, bytes_coder_id, make_coder( common_urns.coders.KV.urn, iter_window_coder_id, make_coder( common_urns.coders.ITERABLE.urn, make_coder(common_urns.coders.KV.urn, window_coder_id, iter_window_coder_id)))) windowed_input_coder_id = make_coder( common_urns.coders.WINDOWED_VALUE.urn, input_coder_id, global_window_coder_id) windowed_output_coder_id = make_coder( common_urns.coders.WINDOWED_VALUE.urn, output_coder_id, global_window_coder_id) self.windowed_input_coder_impl = pipeline_context.coders[ windowed_input_coder_id].get_impl() self.windowed_output_coder_impl = pipeline_context.coders[ windowed_output_coder_id].get_impl() self._bundle_processor_id = self.uid('merge_windows') return beam_fn_api_pb2.ProcessBundleDescriptor( id=self._bundle_processor_id, transforms={ self.TO_SDK_TRANSFORM: beam_runner_api_pb2.PTransform( unique_name='MergeWindows/Read', spec=beam_runner_api_pb2.FunctionSpec( urn=bundle_processor.DATA_INPUT_URN, payload=make_channel_payload(windowed_input_coder_id)), outputs={'input': 'input'}), 'Merge': beam_runner_api_pb2.PTransform( unique_name='MergeWindows/Merge', spec=beam_runner_api_pb2.FunctionSpec( urn=common_urns.primitives.MERGE_WINDOWS.urn, payload=self._windowing_strategy_proto.window_fn. SerializeToString()), inputs={'input': 'input'}, outputs={'output': 'output'}), self.FROM_SDK_TRANSFORM: beam_runner_api_pb2.PTransform( unique_name='MergeWindows/Write', spec=beam_runner_api_pb2.FunctionSpec( urn=bundle_processor.DATA_OUTPUT_URN, payload=make_channel_payload( windowed_output_coder_id)), inputs={'output': 'output'}), }, pcollections={ 'input': beam_runner_api_pb2.PCollection( unique_name='input', windowing_strategy_id=global_windowing_strategy_id, coder_id=input_coder_id), 'output': beam_runner_api_pb2.PCollection( unique_name='output', windowing_strategy_id=global_windowing_strategy_id, coder_id=output_coder_id), }, coders=coders, windowing_strategies={ global_windowing_strategy_id: global_windowing_strategy_proto, }, environments=dict(self._execution_context_ref(). pipeline_components.environments.items()), state_api_service_descriptor=state_api_service_descriptor, timer_api_service_descriptor=data_api_service_descriptor)
def executable_stage_transform( self, known_runner_urns, all_consumers, components): if (len(self.transforms) == 1 and self.transforms[0].spec.urn in known_runner_urns): return self.transforms[0] else: all_inputs = set( pcoll for t in self.transforms for pcoll in t.inputs.values()) all_outputs = set( pcoll for t in self.transforms for pcoll in t.outputs.values()) internal_transforms = set(id(t) for t in self.transforms) external_outputs = [pcoll for pcoll in all_outputs if all_consumers[pcoll] - internal_transforms] stage_components = beam_runner_api_pb2.Components() stage_components.CopyFrom(components) # Only keep the referenced PCollections. for pcoll_id in stage_components.pcollections.keys(): if pcoll_id not in all_inputs and pcoll_id not in all_outputs: del stage_components.pcollections[pcoll_id] # Only keep the transforms in this stage. # Also gather up payload data as we iterate over the transforms. stage_components.transforms.clear() main_inputs = set() side_inputs = [] user_states = [] timers = [] for ix, transform in enumerate(self.transforms): transform_id = 'transform_%d' % ix if transform.spec.urn == common_urns.primitives.PAR_DO.urn: payload = proto_utils.parse_Bytes( transform.spec.payload, beam_runner_api_pb2.ParDoPayload) for tag in payload.side_inputs.keys(): side_inputs.append( beam_runner_api_pb2.ExecutableStagePayload.SideInputId( transform_id=transform_id, local_name=tag)) for tag in payload.state_specs.keys(): user_states.append( beam_runner_api_pb2.ExecutableStagePayload.UserStateId( transform_id=transform_id, local_name=tag)) for tag in payload.timer_specs.keys(): timers.append( beam_runner_api_pb2.ExecutableStagePayload.TimerId( transform_id=transform_id, local_name=tag)) main_inputs.update( pcoll_id for tag, pcoll_id in transform.inputs.items() if tag not in payload.side_inputs) else: main_inputs.update(transform.inputs.values()) stage_components.transforms[transform_id].CopyFrom(transform) main_input_id = only_element(main_inputs - all_outputs) named_inputs = dict({ '%s:%s' % (side.transform_id, side.local_name): stage_components.transforms[side.transform_id].inputs[side.local_name] for side in side_inputs }, main_input=main_input_id) payload = beam_runner_api_pb2.ExecutableStagePayload( environment=components.environments[self.environment], input=main_input_id, outputs=external_outputs, transforms=stage_components.transforms.keys(), components=stage_components, side_inputs=side_inputs, user_states=user_states, timers=timers) return beam_runner_api_pb2.PTransform( unique_name=unique_name(None, self.name), spec=beam_runner_api_pb2.FunctionSpec( urn='beam:runner:executable_stage:v1', payload=payload.SerializeToString()), inputs=named_inputs, outputs={'output_%d' % ix: pcoll for ix, pcoll in enumerate(external_outputs)})
def expand_sdf(stages, context): """Transforms splitable DoFns into pair+split+read.""" for stage in stages: assert len(stage.transforms) == 1 transform = stage.transforms[0] if transform.spec.urn == common_urns.primitives.PAR_DO.urn: pardo_payload = proto_utils.parse_Bytes( transform.spec.payload, beam_runner_api_pb2.ParDoPayload) if pardo_payload.splittable: def copy_like(protos, original, suffix='_copy', **kwargs): if isinstance(original, (str, unicode)): key = original original = protos[original] else: key = 'component' new_id = unique_name(protos, key + suffix) protos[new_id].CopyFrom(original) proto = protos[new_id] for name, value in kwargs.items(): if isinstance(value, dict): getattr(proto, name).clear() getattr(proto, name).update(value) elif isinstance(value, list): del getattr(proto, name)[:] getattr(proto, name).extend(value) elif name == 'urn': proto.spec.urn = value else: setattr(proto, name, value) return new_id def make_stage(base_stage, transform_id, extra_must_follow=()): transform = context.components.transforms[transform_id] return Stage( transform.unique_name, [transform], base_stage.downstream_side_inputs, union(base_stage.must_follow, frozenset(extra_must_follow)), parent=base_stage, environment=base_stage.environment) main_input_tag = only_element(tag for tag in transform.inputs.keys() if tag not in pardo_payload.side_inputs) main_input_id = transform.inputs[main_input_tag] element_coder_id = context.components.pcollections[ main_input_id].coder_id paired_coder_id = context.add_or_get_coder_id( beam_runner_api_pb2.Coder( spec=beam_runner_api_pb2.SdkFunctionSpec( spec=beam_runner_api_pb2.FunctionSpec( urn=common_urns.coders.KV.urn)), component_coder_ids=[element_coder_id, pardo_payload.restriction_coder_id])) paired_pcoll_id = copy_like( context.components.pcollections, main_input_id, '_paired', coder_id=paired_coder_id) pair_transform_id = copy_like( context.components.transforms, transform, unique_name=transform.unique_name + '/PairWithRestriction', urn=common_urns.sdf_components.PAIR_WITH_RESTRICTION.urn, outputs={'out': paired_pcoll_id}) split_pcoll_id = copy_like( context.components.pcollections, main_input_id, '_split', coder_id=paired_coder_id) split_transform_id = copy_like( context.components.transforms, transform, unique_name=transform.unique_name + '/SplitRestriction', urn=common_urns.sdf_components.SPLIT_RESTRICTION.urn, inputs=dict(transform.inputs, **{main_input_tag: paired_pcoll_id}), outputs={'out': split_pcoll_id}) process_transform_id = copy_like( context.components.transforms, transform, unique_name=transform.unique_name + '/Process', urn=common_urns.sdf_components.PROCESS_ELEMENTS.urn, inputs=dict(transform.inputs, **{main_input_tag: split_pcoll_id})) yield make_stage(stage, pair_transform_id) split_stage = make_stage(stage, split_transform_id) yield split_stage yield make_stage( stage, process_transform_id, extra_must_follow=[split_stage]) else: yield stage else: yield stage