def _create_pardo_operation(factory, transform_id, transform_proto, consumers, serialized_fn, side_inputs_proto=None): if side_inputs_proto: input_tags_to_coders = factory.get_input_coders(transform_proto) tagged_side_inputs = [ (tag, beam.pvalue.SideInputData.from_runner_api(si, factory.context)) for tag, si in side_inputs_proto.items() ] tagged_side_inputs.sort(key=lambda tag_si: int( re.match('side([0-9]+)(-.*)?$', tag_si[0]).group(1))) side_input_maps = [ StateBackedSideInputMap(factory.state_handler, transform_id, tag, si, input_tags_to_coders[tag]) for tag, si in tagged_side_inputs ] else: side_input_maps = [] output_tags = list(transform_proto.outputs.keys()) # Hack to match out prefix injected by dataflow runner. def mutate_tag(tag): if 'None' in output_tags: if tag == 'None': return 'out' else: return 'out_' + tag else: return tag dofn_data = pickler.loads(serialized_fn) if not dofn_data[-1]: # Windowing not set. side_input_tags = side_inputs_proto or () pcoll_id, = [ pcoll for tag, pcoll in transform_proto.inputs.items() if tag not in side_input_tags ] windowing = factory.context.windowing_strategies.get_by_id( factory.descriptor.pcollections[pcoll_id].windowing_strategy_id) serialized_fn = pickler.dumps(dofn_data[:-1] + (windowing, )) output_coders = factory.get_output_coders(transform_proto) spec = operation_specs.WorkerDoFn( serialized_fn=serialized_fn, output_tags=[mutate_tag(tag) for tag in output_tags], input=None, side_inputs=None, # Fn API uses proto definitions and the Fn State API output_coders=[output_coders[tag] for tag in output_tags]) return factory.augment_oldstyle_op( operations.DoOperation(transform_proto.unique_name, spec, factory.counter_factory, factory.state_sampler, side_input_maps), transform_proto.unique_name, consumers, output_tags)
def create(factory, transform_id, transform_proto, parameter, consumers): dofn_data = pickler.loads(parameter.value) if len(dofn_data) == 2: # Has side input data. serialized_fn, side_input_data = dofn_data else: # No side input data. serialized_fn, side_input_data = parameter.value, [] def create_side_input(tag, coder): # TODO(robertwb): Extract windows (and keys) out of element data. # TODO(robertwb): Extract state key from ParDoPayload. return operation_specs.WorkerSideInputSource( tag=tag, source=SideInputSource(factory.state_handler, beam_fn_api_pb2.StateKey.MultimapSideInput( key=side_input_tag(transform_id, tag)), coder=coder)) output_tags = list(transform_proto.outputs.keys()) output_coders = factory.get_output_coders(transform_proto) spec = operation_specs.WorkerDoFn( serialized_fn=serialized_fn, output_tags=output_tags, input=None, side_inputs=[ create_side_input(tag, coder) for tag, coder in side_input_data ], output_coders=[output_coders[tag] for tag in output_tags]) return factory.augment_oldstyle_op( operations.DoOperation(transform_proto.unique_name, spec, factory.counter_factory, factory.state_sampler), transform_proto.unique_name, consumers, output_tags)
def _create_pardo_operation( factory, transform_id, transform_proto, consumers, serialized_fn, side_input_data): def create_side_input(tag, coder): # TODO(robertwb): Extract windows (and keys) out of element data. # TODO(robertwb): Extract state key from ParDoPayload. return operation_specs.WorkerSideInputSource( tag=tag, source=SideInputSource( factory.state_handler, beam_fn_api_pb2.StateKey.MultimapSideInput( key=side_input_tag(transform_id, tag)), coder=coder)) output_tags = list(transform_proto.outputs.keys()) # Hack to match out prefix injected by dataflow runner. def mutate_tag(tag): if 'None' in output_tags: if tag == 'None': return 'out' else: return 'out_' + tag else: return tag dofn_data = pickler.loads(serialized_fn) if not dofn_data[-1]: # Windowing not set. pcoll_id, = transform_proto.inputs.values() windowing = factory.context.windowing_strategies.get_by_id( factory.descriptor.pcollections[pcoll_id].windowing_strategy_id) serialized_fn = pickler.dumps(dofn_data[:-1] + (windowing,)) output_coders = factory.get_output_coders(transform_proto) spec = operation_specs.WorkerDoFn( serialized_fn=serialized_fn, output_tags=[mutate_tag(tag) for tag in output_tags], input=None, side_inputs=[ create_side_input(tag, coder) for tag, coder in side_input_data], output_coders=[output_coders[tag] for tag in output_tags]) return factory.augment_oldstyle_op( operations.DoOperation( transform_proto.unique_name, spec, factory.counter_factory, factory.state_sampler), transform_proto.unique_name, consumers, output_tags)
def create_execution_tree(self, descriptor): # TODO(vikasrk): Add an id field to Coder proto and use that instead. coders = {coder.function_spec.id: operation_specs.get_coder_from_spec( json.loads(unpack_function_spec_data(coder.function_spec))) for coder in descriptor.coders} counter_factory = counters.CounterFactory() # TODO(robertwb): Figure out the correct prefix to use for output counters # from StateSampler. state_sampler = statesampler.StateSampler( 'fnapi-step%s-' % descriptor.id, counter_factory) consumers = collections.defaultdict(lambda: collections.defaultdict(list)) ops_by_id = {} reversed_ops = [] for transform in reversed(descriptor.primitive_transform): # TODO(robertwb): Figure out how to plumb through the operation name (e.g. # "s3") from the service through the FnAPI so that msec counters can be # reported and correctly plumbed through the service and the UI. operation_name = 'fnapis%s' % transform.id def only_element(iterable): element, = iterable return element if transform.function_spec.urn == DATA_OUTPUT_URN: target = beam_fn_api_pb2.Target( primitive_transform_reference=transform.id, name=only_element(transform.outputs.keys())) op = DataOutputOperation( operation_name, transform.step_name, consumers[transform.id], counter_factory, state_sampler, coders[only_element(transform.outputs.values()).coder_reference], target, self.data_channel_factory.create_data_channel( transform.function_spec)) elif transform.function_spec.urn == DATA_INPUT_URN: target = beam_fn_api_pb2.Target( primitive_transform_reference=transform.id, name=only_element(transform.inputs.keys())) op = DataInputOperation( operation_name, transform.step_name, consumers[transform.id], counter_factory, state_sampler, coders[only_element(transform.outputs.values()).coder_reference], target, self.data_channel_factory.create_data_channel( transform.function_spec)) elif transform.function_spec.urn == PYTHON_DOFN_URN: def create_side_input(tag, si): # TODO(robertwb): Extract windows (and keys) out of element data. return operation_specs.WorkerSideInputSource( tag=tag, source=SideInputSource( self.state_handler, beam_fn_api_pb2.StateKey( function_spec_reference=si.view_fn.id), coder=unpack_and_deserialize_py_fn(si.view_fn))) output_tags = list(transform.outputs.keys()) spec = operation_specs.WorkerDoFn( serialized_fn=unpack_function_spec_data(transform.function_spec), output_tags=output_tags, input=None, side_inputs=[create_side_input(tag, si) for tag, si in transform.side_inputs.items()], output_coders=[coders[transform.outputs[out].coder_reference] for out in output_tags]) op = operations.DoOperation(operation_name, spec, counter_factory, state_sampler) # TODO(robertwb): Move these to the constructor. op.step_name = transform.step_name for tag, op_consumers in consumers[transform.id].items(): for consumer in op_consumers: op.add_receiver( consumer, output_tags.index(tag)) elif transform.function_spec.urn == IDENTITY_DOFN_URN: op = operations.FlattenOperation(operation_name, None, counter_factory, state_sampler) # TODO(robertwb): Move these to the constructor. op.step_name = transform.step_name for tag, op_consumers in consumers[transform.id].items(): for consumer in op_consumers: op.add_receiver(consumer, 0) elif transform.function_spec.urn == PYTHON_SOURCE_URN: source = load_compressed(unpack_function_spec_data( transform.function_spec)) # TODO(vikasrk): Remove this once custom source is implemented with # splittable dofn via the data plane. spec = operation_specs.WorkerRead( iobase.SourceBundle(1.0, source, None, None), [WindowedValueCoder(source.default_output_coder())]) op = operations.ReadOperation(operation_name, spec, counter_factory, state_sampler) op.step_name = transform.step_name output_tags = list(transform.outputs.keys()) for tag, op_consumers in consumers[transform.id].items(): for consumer in op_consumers: op.add_receiver( consumer, output_tags.index(tag)) else: raise NotImplementedError # Record consumers. for _, inputs in transform.inputs.items(): for target in inputs.target: consumers[target.primitive_transform_reference][target.name].append( op) reversed_ops.append(op) ops_by_id[transform.id] = op return list(reversed(reversed_ops)), ops_by_id