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, 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 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, any_param=proto_utils.pack_Any( wrappers_pb2.BytesValue( value=param)), 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, any_param=proto_utils.pack_Any( wrappers_pb2.BytesValue( value=param)), payload=param))], downstream_side_inputs=frozenset(), must_follow=union(frozenset(flatten_writes), stage.must_follow)) else: yield stage
def to_runner_api_parameter(self, context): return (_StreamingGroupAlsoByWindow.urn, wrappers_pb2.BytesValue( value=context.windowing_strategies.get_id(self.windowing)))
def to_runner_api_parameter(self, context): return (python_urns.PICKLED_TRANSFORM, wrappers_pb2.BytesValue(value=pickler.dumps(self)))
def to_runner_api_parameter(self, context): return (urns.PICKLED_WINDOW_FN, wrappers_pb2.BytesValue(value=pickler.dumps(self)))
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'])) } def only_element(iterable): element, = iterable return element 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=sdk_worker.DATA_OUTPUT_URN, parameter=proto_utils.pack_Any(data_operation_spec)) 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=sdk_worker.DATA_INPUT_URN, parameter=proto_utils.pack_Any(data_operation_spec)) 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. transform_spec = beam_runner_api_pb2.FunctionSpec( urn=sdk_worker.PYTHON_SOURCE_URN, parameter=proto_utils.pack_Any( wrappers_pb2.BytesValue(value=base64.b64decode( pickler.dumps(operation.source.source))))) 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[sdk_worker.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=sdk_worker.PYTHON_DOFN_URN, parameter=proto_utils.pack_Any( wrappers_pb2.BytesValue( value=augmented_serialized_fn))) elif isinstance(operation, operation_specs.WorkerFlatten): # Flatten is nice and simple. transform_spec = beam_runner_api_pb2.FunctionSpec( urn=sdk_worker.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, codersyyy=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 to_runner_api_parameter(self, context): return (python_urns.PICKLED_CODER, wrappers_pb2.BytesValue(value=serialize_coder(self)), ())
def unpack_function_spec_data(function_spec): """Returns unpacked data from function spec proto.""" data = wrappers_pb2.BytesValue() function_spec.data.Unpack(data) return data.value
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 = proto_utils.pack_Any( wrappers_pb2.BytesValue(value=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 = set( 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, parameter=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, parameter=pcoll_as_param)) ], must_follow={write_pcoll}) fuse(read_pcoll, consumer) # 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