def process_bundle(self, inputs, expected_outputs): # Unique id for the instruction processing this bundle. BundleManager._uid_counter += 1 process_bundle_id = 'bundle_%s' % BundleManager._uid_counter # Register the bundle descriptor, if needed. if self._registered: registration_future = None else: process_bundle_registration = beam_fn_api_pb2.InstructionRequest( register=beam_fn_api_pb2.RegisterRequest( process_bundle_descriptor=[self._bundle_descriptor])) registration_future = self._controller.control_handler.push( process_bundle_registration) self._registered = True # Write all the input data to the channel. for (transform_id, name), elements in inputs.items(): data_out = self._controller.data_plane_handler.output_stream( process_bundle_id, beam_fn_api_pb2.Target( primitive_transform_reference=transform_id, name=name)) for element_data in elements: data_out.write(element_data) data_out.close() # Actually start the bundle. if registration_future and registration_future.get().error: raise RuntimeError(registration_future.get().error) process_bundle = beam_fn_api_pb2.InstructionRequest( instruction_id=process_bundle_id, process_bundle=beam_fn_api_pb2.ProcessBundleRequest( process_bundle_descriptor_reference=self._bundle_descriptor.id)) result_future = self._controller.control_handler.push(process_bundle) with ProgressRequester( self._controller, process_bundle_id, self._progress_frequency): # Gather all output data. expected_targets = [ beam_fn_api_pb2.Target(primitive_transform_reference=transform_id, name=output_name) for (transform_id, output_name), _ in expected_outputs.items()] logging.debug('Gather all output data from %s.', expected_targets) for output in self._controller.data_plane_handler.input_elements( process_bundle_id, expected_targets, abort_callback=lambda: (result_future.is_done() and result_future.get().error)): target_tuple = ( output.target.primitive_transform_reference, output.target.name) if target_tuple in expected_outputs: self._get_buffer(expected_outputs[target_tuple]).append(output.data) logging.debug('Wait for the bundle to finish.') result = result_future.get() if result.error: raise RuntimeError(result.error) return result
def test_inactive_bundle_processor_returns_empty_split_response(self): bundle_processor = mock.MagicMock() bundle_processor_cache = BundleProcessorCache(None, None, {}) bundle_processor_cache.activate('instruction_id') worker = SdkWorker(bundle_processor_cache) split_request = beam_fn_api_pb2.InstructionRequest( instruction_id='split_instruction_id', process_bundle_split=beam_fn_api_pb2.ProcessBundleSplitRequest( instruction_id='instruction_id')) self.assertEqual( worker.do_instruction(split_request), beam_fn_api_pb2.InstructionResponse( instruction_id='split_instruction_id', process_bundle_split=beam_fn_api_pb2. ProcessBundleSplitResponse())) # Add a mock bundle processor as if it was running before it's released bundle_processor_cache.active_bundle_processors['instruction_id'] = ( 'descriptor_id', bundle_processor) bundle_processor_cache.release('instruction_id') self.assertEqual( worker.do_instruction(split_request), beam_fn_api_pb2.InstructionResponse( instruction_id='split_instruction_id', process_bundle_split=beam_fn_api_pb2. ProcessBundleSplitResponse()))
def _check_fn_registration_multi_request(self, *args): """Check the function registration calls to the sdk_harness. Args: tuple of request_count, number of process_bundles per request and workers counts to process the request. """ for (request_count, process_bundles_per_request) in args: requests = [] process_bundle_descriptors = [] for i in range(request_count): pbd = self._get_process_bundles(i, process_bundles_per_request) process_bundle_descriptors.extend(pbd) requests.append( beam_fn_api_pb2.InstructionRequest( instruction_id=str(i), register=beam_fn_api_pb2.RegisterRequest( process_bundle_descriptor=process_bundle_descriptors))) test_controller = BeamFnControlServicer(requests) server = grpc.server(UnboundedThreadPoolExecutor()) beam_fn_api_pb2_grpc.add_BeamFnControlServicer_to_server( test_controller, server) test_port = server.add_insecure_port("[::]:0") server.start() harness = sdk_worker.SdkHarness( "localhost:%s" % test_port, state_cache_size=100) harness.run() self.assertEqual(harness._bundle_processor_cache.fns, {item.id: item for item in process_bundle_descriptors})
def merge(self, merge_context): # type: (window.WindowFn.MergeContext) -> None worker_handler = self.worker_handle() assert self.windowed_input_coder_impl is not None assert self.windowed_output_coder_impl is not None process_bundle_id = self.uid('process') to_worker = worker_handler.data_conn.output_stream( process_bundle_id, self.TO_SDK_TRANSFORM) to_worker.write( self.windowed_input_coder_impl.encode_nested( window.GlobalWindows.windowed_value( (b'', merge_context.windows)))) to_worker.close() process_bundle_req = beam_fn_api_pb2.InstructionRequest( instruction_id=process_bundle_id, process_bundle=beam_fn_api_pb2.ProcessBundleRequest( process_bundle_descriptor_id=self._bundle_processor_id)) result_future = worker_handler.control_conn.push(process_bundle_req) for output in worker_handler.data_conn.input_elements( process_bundle_id, [self.FROM_SDK_TRANSFORM], abort_callback=lambda: bool(result_future.is_done() and result_future.get().error)): if isinstance(output, beam_fn_api_pb2.Elements.Data): windowed_result = self.windowed_output_coder_impl.decode_nested( output.data) for merge_result, originals in windowed_result.value[1][1]: merge_context.merge(originals, merge_result) else: raise RuntimeError("Unexpected data: %s" % output) result = result_future.get() if result.error: raise RuntimeError(result.error)
def test_fn_registration(self): fns = [beam_fn_api_pb2.FunctionSpec(id=str(ix)) for ix in range(4)] process_bundle_descriptors = [ beam_fn_api_pb2.ProcessBundleDescriptor( id=str(100 + ix), primitive_transform=[ beam_fn_api_pb2.PrimitiveTransform(function_spec=fn) ]) for ix, fn in enumerate(fns) ] test_controller = BeamFnControlServicer([ beam_fn_api_pb2.InstructionRequest( register=beam_fn_api_pb2.RegisterRequest( process_bundle_descriptor=process_bundle_descriptors)) ]) server = grpc.server(futures.ThreadPoolExecutor(max_workers=10)) beam_fn_api_pb2.add_BeamFnControlServicer_to_server( test_controller, server) test_port = server.add_insecure_port("[::]:0") server.start() channel = grpc.insecure_channel("localhost:%s" % test_port) harness = sdk_worker.SdkHarness(channel) harness.run() self.assertEqual( harness.worker.fns, {item.id: item for item in fns + process_bundle_descriptors})
def test_fn_registration(self): process_bundle_descriptors = [ beam_fn_api_pb2.ProcessBundleDescriptor( id=str(100 + ix), transforms={ str(ix): beam_runner_api_pb2.PTransform(unique_name=str(ix)) }) for ix in range(4) ] test_controller = BeamFnControlServicer([ beam_fn_api_pb2.InstructionRequest( register=beam_fn_api_pb2.RegisterRequest( process_bundle_descriptor=process_bundle_descriptors)) ]) server = grpc.server(futures.ThreadPoolExecutor(max_workers=10)) beam_fn_api_pb2.add_BeamFnControlServicer_to_server( test_controller, server) test_port = server.add_insecure_port("[::]:0") server.start() channel = grpc.insecure_channel("localhost:%s" % test_port) harness = sdk_worker.SdkHarness(channel) harness.run() self.assertEqual( harness.worker.fns, {item.id: item for item in process_bundle_descriptors})
def test_source_split_via_instruction(self): source = RangeSource(0, 100) expected_splits = list(source.split(30)) test_controller = BeamFnControlServicer([ beam_fn_api_pb2.InstructionRequest( instruction_id="register_request", register=beam_fn_api_pb2.RegisterRequest( process_bundle_descriptor=[ beam_fn_api_pb2.ProcessBundleDescriptor( primitive_transform=[ beam_fn_api_pb2.PrimitiveTransform( function_spec=sdk_harness. serialize_and_pack_py_fn( SourceBundle(1.0, source, None, None), sdk_harness.PYTHON_SOURCE_URN, id="src")) ]) ])), beam_fn_api_pb2.InstructionRequest( instruction_id="split_request", initial_source_split=beam_fn_api_pb2.InitialSourceSplitRequest( desired_bundle_size_bytes=30, source_reference="src")) ]) server = grpc.server(futures.ThreadPoolExecutor(max_workers=10)) beam_fn_api_pb2.add_BeamFnControlServicer_to_server( test_controller, server) test_port = server.add_insecure_port("[::]:0") server.start() channel = grpc.insecure_channel("localhost:%s" % test_port) harness = sdk_harness.SdkHarness(channel) harness.run() split_response = test_controller.responses[ "split_request"].initial_source_split self.assertEqual(expected_splits, [ sdk_harness.unpack_and_deserialize_py_fn(s.source) for s in split_response.splits ]) self.assertEqual([s.weight for s in expected_splits], [s.relative_size for s in split_response.splits])
def test_harness_monitoring_infos_and_metadata(self): # Clear the process wide metric container. MetricsEnvironment.process_wide_container().reset() # Create a process_wide metric. urn = 'my.custom.urn' labels = {'key': 'value'} InternalMetrics.counter(urn=urn, labels=labels, process_wide=True).inc(10) harness_monitoring_infos_request = beam_fn_api_pb2.InstructionRequest( instruction_id="monitoring_infos", harness_monitoring_infos=beam_fn_api_pb2. HarnessMonitoringInfosRequest()) responses = self.get_responses([harness_monitoring_infos_request]) expected_monitoring_info = monitoring_infos.int64_counter( urn, 10, labels=labels) monitoring_data = (responses['monitoring_infos']. harness_monitoring_infos.monitoring_data) # Request the full MonitoringInfo metadata for the returned short_ids. short_ids = list(monitoring_data.keys()) monitoring_infos_metadata_request = beam_fn_api_pb2.InstructionRequest( instruction_id="monitoring_infos_metadata", monitoring_infos=beam_fn_api_pb2.MonitoringInfosMetadataRequest( monitoring_info_id=short_ids)) responses = self.get_responses([monitoring_infos_metadata_request]) # Request the full MonitoringInfo metadata to be returned now. expected_monitoring_info.ClearField("payload") # Verify that one of the returned monitoring infos is our expected # monitoring info. short_id_to_mi = (responses['monitoring_infos_metadata']. monitoring_infos.monitoring_info) found = False for mi in short_id_to_mi.values(): # Clear the timestamp before comparing mi.ClearField("start_time") if mi == expected_monitoring_info: found = True self.assertTrue(found, str(responses['monitoring_infos_metadata']))
def _run_map_task(self, map_task, control_handler, state_handler, data_plane_handler, data_operation_spec): registration, sinks, input_data = self._map_task_registration( map_task, state_handler, data_operation_spec) control_handler.push(registration) process_bundle = beam_fn_api_pb2.InstructionRequest( instruction_id=self._next_uid(), process_bundle=beam_fn_api_pb2.ProcessBundleRequest( process_bundle_descriptor_reference=registration.register. process_bundle_descriptor[0].id)) for (transform_id, name), elements in input_data.items(): data_out = data_plane_handler.output_stream( process_bundle.instruction_id, beam_fn_api_pb2.Target( primitive_transform_reference=transform_id, name=name)) data_out.write(elements) data_out.close() control_handler.push(process_bundle) while True: result = control_handler.pull() if result.instruction_id == process_bundle.instruction_id: if result.error: raise RuntimeError(result.error) expected_targets = [ beam_fn_api_pb2.Target( primitive_transform_reference=transform_id, name=output_name) for (transform_id, output_name), _ in sinks.items() ] for output in data_plane_handler.input_elements( process_bundle.instruction_id, expected_targets): target_tuple = ( output.target.primitive_transform_reference, output.target.name) if target_tuple not in sinks: # Unconsumed output. continue sink_op = sinks[target_tuple] coder = sink_op.output_coders[0] input_stream = create_InputStream(output.data) elements = [] while input_stream.size() > 0: elements.append(coder.get_impl().decode_from_stream( input_stream, True)) if not sink_op.write_windowed_values: elements = [e.value for e in elements] for e in elements: sink_op.output_buffer.append(e) return
def _register_bundle_descriptor(self): # type: () -> Optional[ControlFuture] if self._registered: registration_future = None else: assert self._worker_handler is not None process_bundle_registration = beam_fn_api_pb2.InstructionRequest( register=beam_fn_api_pb2.RegisterRequest( process_bundle_descriptor=[self._bundle_descriptor])) registration_future = self._worker_handler.control_conn.push( process_bundle_registration) self._registered = True return registration_future
def run(self): while not self._done: try: progress_result = self._worker_handler.control_conn.push( beam_fn_api_pb2.InstructionRequest( process_bundle_progress=beam_fn_api_pb2. ProcessBundleProgressRequest( instruction_id=self._instruction_id))).get() self._latest_progress = progress_result.process_bundle_progress if self._callback: self._callback(self._latest_progress) except Exception as exn: _LOGGER.error("Bad progress: %s", exn) time.sleep(self._frequency)
def _map_task_registration(self, map_task, state_handler, data_operation_spec): input_data, side_input_data, runner_sinks, process_bundle_descriptor = ( self._map_task_to_protos(map_task, data_operation_spec)) # Side inputs will be accessed over the state API. for key, elements_data in side_input_data.items(): state_key = beam_fn_api_pb2.StateKey.MultimapSideInput(key=key) state_handler.Clear(state_key) state_handler.Append(state_key, [elements_data]) return beam_fn_api_pb2.InstructionRequest( instruction_id=self._next_uid(), register=beam_fn_api_pb2.RegisterRequest( process_bundle_descriptor=[process_bundle_descriptor]) ), runner_sinks, input_data
def test_failed_bundle_processor_returns_failed_split_response(self): bundle_processor = mock.MagicMock() bundle_processor_cache = BundleProcessorCache(None, None, {}) bundle_processor_cache.activate('instruction_id') worker = SdkWorker(bundle_processor_cache) # Add a mock bundle processor as if it was running before it's discarded bundle_processor_cache.active_bundle_processors['instruction_id'] = ( 'descriptor_id', bundle_processor) bundle_processor_cache.discard('instruction_id') split_request = beam_fn_api_pb2.InstructionRequest( instruction_id='split_instruction_id', process_bundle_split=beam_fn_api_pb2.ProcessBundleSplitRequest( instruction_id='instruction_id')) hc.assert_that( worker.do_instruction(split_request).error, hc.contains_string( 'Bundle processing associated with instruction_id has failed'))
def run_stage(self, controller, pipeline_components, stage, pcoll_buffers, safe_coders): context = pipeline_context.PipelineContext(pipeline_components) data_operation_spec = controller.data_operation_spec() def extract_endpoints(stage): # Returns maps of transform names to PCollection identifiers. # Also mutates IO stages to point to the data data_operation_spec. data_input = {} data_side_input = {} data_output = {} for transform in stage.transforms: if transform.spec.urn in (bundle_processor.DATA_INPUT_URN, bundle_processor.DATA_OUTPUT_URN): pcoll_id = transform.spec.payload if transform.spec.urn == bundle_processor.DATA_INPUT_URN: target = transform.unique_name, only_element( transform.outputs) data_input[target] = pcoll_id elif transform.spec.urn == bundle_processor.DATA_OUTPUT_URN: target = transform.unique_name, only_element( transform.inputs) data_output[target] = pcoll_id else: raise NotImplementedError if data_operation_spec: transform.spec.payload = data_operation_spec.SerializeToString( ) else: transform.spec.payload = "" elif transform.spec.urn == urns.PARDO_TRANSFORM: payload = proto_utils.parse_Bytes( transform.spec.payload, beam_runner_api_pb2.ParDoPayload) for tag, si in payload.side_inputs.items(): data_side_input[transform.unique_name, tag] = ( 'materialize:' + transform.inputs[tag], beam.pvalue.SideInputData.from_runner_api( si, None)) return data_input, data_side_input, data_output logging.info('Running %s', stage.name) logging.debug(' %s', stage) data_input, data_side_input, data_output = extract_endpoints(stage) process_bundle_descriptor = beam_fn_api_pb2.ProcessBundleDescriptor( id=self._next_uid(), transforms={ transform.unique_name: transform for transform in stage.transforms }, pcollections=dict(pipeline_components.pcollections.items()), coders=dict(pipeline_components.coders.items()), windowing_strategies=dict( pipeline_components.windowing_strategies.items()), environments=dict(pipeline_components.environments.items())) process_bundle_registration = beam_fn_api_pb2.InstructionRequest( instruction_id=self._next_uid(), register=beam_fn_api_pb2.RegisterRequest( process_bundle_descriptor=[process_bundle_descriptor])) process_bundle = beam_fn_api_pb2.InstructionRequest( instruction_id=self._next_uid(), process_bundle=beam_fn_api_pb2.ProcessBundleRequest( process_bundle_descriptor_reference=process_bundle_descriptor. id)) # Write all the input data to the channel. for (transform_id, name), pcoll_id in data_input.items(): data_out = controller.data_plane_handler.output_stream( process_bundle.instruction_id, beam_fn_api_pb2.Target( primitive_transform_reference=transform_id, name=name)) for element_data in pcoll_buffers[pcoll_id]: data_out.write(element_data) data_out.close() # Store the required side inputs into state. for (transform_id, tag), (pcoll_id, si) in data_side_input.items(): elements_by_window = _WindowGroupingBuffer(si) for element_data in pcoll_buffers[pcoll_id]: elements_by_window.append(element_data) for window, elements_data in elements_by_window.items(): state_key = beam_fn_api_pb2.StateKey( multimap_side_input=beam_fn_api_pb2.StateKey. MultimapSideInput(ptransform_id=transform_id, side_input_id=tag, window=window)) controller.state_handler.blocking_append( state_key, elements_data, process_bundle.instruction_id) # Register and start running the bundle. logging.debug('Register and start running the bundle') controller.control_handler.push(process_bundle_registration) controller.control_handler.push(process_bundle) # Wait for the bundle to finish. logging.debug('Wait for the bundle to finish.') while True: result = controller.control_handler.pull() if result and result.instruction_id == process_bundle.instruction_id: if result.error: raise RuntimeError(result.error) break expected_targets = [ beam_fn_api_pb2.Target(primitive_transform_reference=transform_id, name=output_name) for (transform_id, output_name), _ in data_output.items() ] # Gather all output data. logging.debug('Gather all output data from %s.', expected_targets) for output in controller.data_plane_handler.input_elements( process_bundle.instruction_id, expected_targets): target_tuple = (output.target.primitive_transform_reference, output.target.name) if target_tuple in data_output: pcoll_id = data_output[target_tuple] if pcoll_id.startswith('materialize:'): # Just store the data chunks for replay. pcoll_buffers[pcoll_id].append(output.data) elif pcoll_id.startswith('group:'): # This is a grouping write, create a grouping buffer if needed. if pcoll_id not in pcoll_buffers: original_gbk_transform = pcoll_id.split(':', 1)[1] transform_proto = pipeline_components.transforms[ original_gbk_transform] input_pcoll = only_element( transform_proto.inputs.values()) output_pcoll = only_element( transform_proto.outputs.values()) pre_gbk_coder = context.coders[ safe_coders[pipeline_components. pcollections[input_pcoll].coder_id]] post_gbk_coder = context.coders[ safe_coders[pipeline_components. pcollections[output_pcoll].coder_id]] windowing_strategy = context.windowing_strategies[ pipeline_components.pcollections[output_pcoll]. windowing_strategy_id] pcoll_buffers[pcoll_id] = _GroupingBuffer( pre_gbk_coder, post_gbk_coder, windowing_strategy) pcoll_buffers[pcoll_id].append(output.data) else: # These should be the only two identifiers we produce for now, # but special side input writes may go here. raise NotImplementedError(pcoll_id) return result
def _map_task_registration(self, map_task, state_handler, data_operation_spec): input_data = {} runner_sinks = {} transforms = [] transform_index_to_id = {} # Maps coders to new coder objects and references. coders = {} def coder_id(coder): if coder not in coders: coders[coder] = beam_fn_api_pb2.Coder( function_spec=sdk_worker.pack_function_spec_data( json.dumps(coder.as_cloud_object()), sdk_worker.PYTHON_CODER_URN, id=self._next_uid())) return coders[coder].function_spec.id def output_tags(op): return getattr(op, 'output_tags', ['out']) def as_target(op_input): input_op_index, input_output_index = op_input input_op = map_task[input_op_index][1] return { 'ignored_input_tag': beam_fn_api_pb2.Target.List(target=[ beam_fn_api_pb2.Target( primitive_transform_reference=transform_index_to_id[ input_op_index], name=output_tags(input_op)[input_output_index]) ]) } def outputs(op): return { tag: beam_fn_api_pb2.PCollection(coder_reference=coder_id(coder)) for tag, coder in zip(output_tags(op), op.output_coders) } for op_ix, (stage_name, operation) in enumerate(map_task): transform_id = transform_index_to_id[op_ix] = self._next_uid() if isinstance(operation, operation_specs.WorkerInMemoryWrite): # Write this data back to the runner. fn = beam_fn_api_pb2.FunctionSpec( urn=sdk_worker.DATA_OUTPUT_URN, id=self._next_uid()) if data_operation_spec: fn.data.Pack(data_operation_spec) inputs = as_target(operation.input) side_inputs = {} runner_sinks[(transform_id, 'out')] = operation elif isinstance(operation, operation_specs.WorkerRead): # A Read is either translated to a direct injection of windowed values # into the sdk worker, or an injection of the source object into the # sdk worker as data followed by an SDF that reads that source. if (isinstance(operation.source.source, maptask_executor_runner.InMemorySource) and isinstance( operation.source.source.default_output_coder(), WindowedValueCoder)): output_stream = create_OutputStream() element_coder = (operation.source.source. default_output_coder().get_impl()) # Re-encode the elements in the nested context and # concatenate them together for element in operation.source.source.read(None): element_coder.encode_to_stream(element, output_stream, True) target_name = self._next_uid() input_data[(transform_id, target_name)] = output_stream.get() fn = beam_fn_api_pb2.FunctionSpec( urn=sdk_worker.DATA_INPUT_URN, id=self._next_uid()) if data_operation_spec: fn.data.Pack(data_operation_spec) inputs = {target_name: beam_fn_api_pb2.Target.List()} side_inputs = {} else: # Read the source object from the runner. source_coder = beam.coders.DillCoder() input_transform_id = self._next_uid() output_stream = create_OutputStream() source_coder.get_impl().encode_to_stream( GlobalWindows.windowed_value(operation.source), output_stream, True) target_name = self._next_uid() input_data[(input_transform_id, target_name)] = output_stream.get() input_ptransform = beam_fn_api_pb2.PrimitiveTransform( id=input_transform_id, function_spec=beam_fn_api_pb2.FunctionSpec( urn=sdk_worker.DATA_INPUT_URN, id=self._next_uid()), # TODO(robertwb): Possible name collision. step_name=stage_name + '/inject_source', inputs={target_name: beam_fn_api_pb2.Target.List()}, outputs={ 'out': beam_fn_api_pb2.PCollection( coder_reference=coder_id(source_coder)) }) if data_operation_spec: input_ptransform.function_spec.data.Pack( data_operation_spec) transforms.append(input_ptransform) # Read the elements out of the source. fn = sdk_worker.pack_function_spec_data( OLDE_SOURCE_SPLITTABLE_DOFN_DATA, sdk_worker.PYTHON_DOFN_URN, id=self._next_uid()) inputs = { 'ignored_input_tag': beam_fn_api_pb2.Target.List(target=[ beam_fn_api_pb2.Target( primitive_transform_reference= input_transform_id, name='out') ]) } side_inputs = {} elif isinstance(operation, operation_specs.WorkerDoFn): fn = sdk_worker.pack_function_spec_data( operation.serialized_fn, sdk_worker.PYTHON_DOFN_URN, id=self._next_uid()) inputs = as_target(operation.input) # Store the contents of each side input for state access. for si in operation.side_inputs: assert isinstance(si.source, iobase.BoundedSource) element_coder = si.source.default_output_coder() view_id = self._next_uid() # TODO(robertwb): Actually flesh out the ViewFn API. side_inputs[si.tag] = beam_fn_api_pb2.SideInput( view_fn=sdk_worker.serialize_and_pack_py_fn( element_coder, urn=sdk_worker.PYTHON_ITERABLE_VIEWFN_URN, id=view_id)) # Re-encode the elements in the nested context and # concatenate them together output_stream = create_OutputStream() for element in si.source.read( si.source.get_range_tracker(None, None)): element_coder.get_impl().encode_to_stream( element, output_stream, True) elements_data = output_stream.get() state_key = beam_fn_api_pb2.StateKey.MultimapSideInput( key=view_id) state_handler.Clear(state_key) state_handler.Append(state_key, elements_data) elif isinstance(operation, operation_specs.WorkerFlatten): fn = sdk_worker.pack_function_spec_data( operation.serialized_fn, sdk_worker.IDENTITY_DOFN_URN, id=self._next_uid()) inputs = { 'ignored_input_tag': beam_fn_api_pb2.Target.List(target=[ beam_fn_api_pb2.Target( primitive_transform_reference= transform_index_to_id[input_op_index], name=output_tags(map_task[input_op_index] [1])[input_output_index]) for input_op_index, input_output_index in operation.inputs ]) } side_inputs = {} else: raise TypeError(operation) ptransform = beam_fn_api_pb2.PrimitiveTransform( id=transform_id, function_spec=fn, step_name=stage_name, inputs=inputs, side_inputs=side_inputs, outputs=outputs(operation)) transforms.append(ptransform) process_bundle_descriptor = beam_fn_api_pb2.ProcessBundleDescriptor( id=self._next_uid(), coders=coders.values(), primitive_transform=transforms) return beam_fn_api_pb2.InstructionRequest( instruction_id=self._next_uid(), register=beam_fn_api_pb2.RegisterRequest( process_bundle_descriptor=[process_bundle_descriptor ])), runner_sinks, input_data
def process_bundle( self, inputs, # type: Mapping[str, execution.PartitionableBuffer] expected_outputs, # type: DataOutput fired_timers, # type: Mapping[Tuple[str, str], execution.PartitionableBuffer] expected_output_timers, # type: Dict[Tuple[str, str], str] dry_run=False, ): # type: (...) -> BundleProcessResult # Unique id for the instruction processing this bundle. with BundleManager._lock: BundleManager._uid_counter += 1 process_bundle_id = 'bundle_%s' % BundleManager._uid_counter self._worker_handler = self.bundle_context_manager.worker_handlers[ BundleManager._uid_counter % len(self.bundle_context_manager.worker_handlers)] split_manager = self._select_split_manager() if not split_manager: # Send timers. for transform_id, timer_family_id in expected_output_timers.keys(): self._send_timers_to_worker( process_bundle_id, transform_id, timer_family_id, fired_timers.get((transform_id, timer_family_id), [])) # If there is no split_manager, write all input data to the channel. for transform_id, elements in inputs.items(): self._send_input_to_worker(process_bundle_id, transform_id, elements) # Actually start the bundle. process_bundle_req = beam_fn_api_pb2.InstructionRequest( instruction_id=process_bundle_id, process_bundle=beam_fn_api_pb2.ProcessBundleRequest( process_bundle_descriptor_id=self.bundle_context_manager. process_bundle_descriptor.id, cache_tokens=[next(self._cache_token_generator)])) result_future = self._worker_handler.control_conn.push( process_bundle_req) split_results = [ ] # type: List[beam_fn_api_pb2.ProcessBundleSplitResponse] with ProgressRequester(self._worker_handler, process_bundle_id, self._progress_frequency): if split_manager: split_results = self._generate_splits_for_testing( split_manager, inputs, process_bundle_id) expect_reads = list(expected_outputs.keys()) expect_reads.extend(list(expected_output_timers.keys())) # Gather all output data. for output in self._worker_handler.data_conn.input_elements( process_bundle_id, expect_reads, abort_callback=lambda: (result_future.is_done() and result_future.get().error)): if isinstance(output, beam_fn_api_pb2.Elements.Timers) and not dry_run: with BundleManager._lock: timer_buffer = self.bundle_context_manager.get_buffer( expected_output_timers[(output.transform_id, output.timer_family_id)], output.transform_id) if timer_buffer.cleared: timer_buffer.reset() timer_buffer.append(output.timers) if isinstance(output, beam_fn_api_pb2.Elements.Data) and not dry_run: with BundleManager._lock: self.bundle_context_manager.get_buffer( expected_outputs[output.transform_id], output.transform_id).append(output.data) _LOGGER.debug('Wait for the bundle %s to finish.' % process_bundle_id) result = result_future.get( ) # type: beam_fn_api_pb2.InstructionResponse if result.error: raise RuntimeError(result.error) if result.process_bundle.requires_finalization: finalize_request = beam_fn_api_pb2.InstructionRequest( finalize_bundle=beam_fn_api_pb2.FinalizeBundleRequest( instruction_id=process_bundle_id)) self._worker_handler.control_conn.push(finalize_request) return result, split_results
def _generate_splits_for_testing( self, split_manager, inputs, # type: Mapping[str, PartitionableBuffer] process_bundle_id): # type: (...) -> List[beam_fn_api_pb2.ProcessBundleSplitResponse] split_results = [ ] # type: List[beam_fn_api_pb2.ProcessBundleSplitResponse] read_transform_id, buffer_data = only_element(inputs.items()) byte_stream = b''.join(buffer_data) num_elements = len( list( self.bundle_context_manager.get_input_coder_impl( read_transform_id).decode_all(byte_stream))) # Start the split manager in case it wants to set any breakpoints. split_manager_generator = split_manager(num_elements) try: split_fraction = next(split_manager_generator) done = False except StopIteration: done = True # Send all the data. self._send_input_to_worker(process_bundle_id, read_transform_id, [byte_stream]) assert self._worker_handler is not None # Execute the requested splits. while not done: if split_fraction is None: split_result = None else: split_request = beam_fn_api_pb2.InstructionRequest( process_bundle_split=beam_fn_api_pb2. ProcessBundleSplitRequest( instruction_id=process_bundle_id, desired_splits={ read_transform_id: beam_fn_api_pb2.ProcessBundleSplitRequest. DesiredSplit(fraction_of_remainder=split_fraction, estimated_input_elements=num_elements) })) split_response = self._worker_handler.control_conn.push( split_request).get( ) # type: beam_fn_api_pb2.InstructionResponse for t in (0.05, 0.1, 0.2): waiting = ('Instruction not running', 'not yet scheduled') if any(msg in split_response.error for msg in waiting): time.sleep(t) split_response = self._worker_handler.control_conn.push( split_request).get() if 'Unknown process bundle' in split_response.error: # It may have finished too fast. split_result = None elif split_response.error: raise RuntimeError(split_response.error) else: split_result = split_response.process_bundle_split split_results.append(split_result) try: split_fraction = split_manager_generator.send(split_result) except StopIteration: break return split_results
def run_stage( self, controller, pipeline_components, stage, pcoll_buffers, safe_coders): coders = pipeline_context.PipelineContext(pipeline_components).coders data_operation_spec = controller.data_operation_spec() def extract_endpoints(stage): # Returns maps of transform names to PCollection identifiers. # Also mutates IO stages to point to the data data_operation_spec. data_input = {} data_side_input = {} data_output = {} for transform in stage.transforms: pcoll_id = transform.spec.payload if transform.spec.urn in (bundle_processor.DATA_INPUT_URN, bundle_processor.DATA_OUTPUT_URN): if transform.spec.urn == bundle_processor.DATA_INPUT_URN: target = transform.unique_name, only_element(transform.outputs) data_input[target] = pcoll_id elif transform.spec.urn == bundle_processor.DATA_OUTPUT_URN: target = transform.unique_name, only_element(transform.inputs) data_output[target] = pcoll_id else: raise NotImplementedError if data_operation_spec: transform.spec.payload = data_operation_spec.SerializeToString() transform.spec.any_param.Pack(data_operation_spec) else: transform.spec.payload = "" transform.spec.any_param.Clear() return data_input, data_side_input, data_output logging.info('Running %s', stage.name) logging.debug(' %s', stage) data_input, data_side_input, data_output = extract_endpoints(stage) if data_side_input: raise NotImplementedError('Side inputs.') process_bundle_descriptor = beam_fn_api_pb2.ProcessBundleDescriptor( id=self._next_uid(), transforms={transform.unique_name: transform for transform in stage.transforms}, pcollections=dict(pipeline_components.pcollections.items()), coders=dict(pipeline_components.coders.items()), windowing_strategies=dict( pipeline_components.windowing_strategies.items()), environments=dict(pipeline_components.environments.items())) process_bundle_registration = beam_fn_api_pb2.InstructionRequest( instruction_id=self._next_uid(), register=beam_fn_api_pb2.RegisterRequest( process_bundle_descriptor=[process_bundle_descriptor])) process_bundle = beam_fn_api_pb2.InstructionRequest( instruction_id=self._next_uid(), process_bundle=beam_fn_api_pb2.ProcessBundleRequest( process_bundle_descriptor_reference= process_bundle_descriptor.id)) # Write all the input data to the channel. for (transform_id, name), pcoll_id in data_input.items(): data_out = controller.data_plane_handler.output_stream( process_bundle.instruction_id, beam_fn_api_pb2.Target( primitive_transform_reference=transform_id, name=name)) for element_data in pcoll_buffers[pcoll_id]: data_out.write(element_data) data_out.close() # Register and start running the bundle. controller.control_handler.push(process_bundle_registration) controller.control_handler.push(process_bundle) # Wait for the bundle to finish. while True: result = controller.control_handler.pull() if result and result.instruction_id == process_bundle.instruction_id: if result.error: raise RuntimeError(result.error) break # Gather all output data. expected_targets = [ beam_fn_api_pb2.Target(primitive_transform_reference=transform_id, name=output_name) for (transform_id, output_name), _ in data_output.items()] for output in controller.data_plane_handler.input_elements( process_bundle.instruction_id, expected_targets): target_tuple = ( output.target.primitive_transform_reference, output.target.name) if target_tuple in data_output: pcoll_id = data_output[target_tuple] if pcoll_id.startswith('materialize:'): # Just store the data chunks for replay. pcoll_buffers[pcoll_id].append(output.data) elif pcoll_id.startswith('group:'): # This is a grouping write, create a grouping buffer if needed. if pcoll_id not in pcoll_buffers: original_gbk_transform = pcoll_id.split(':', 1)[1] transform_proto = pipeline_components.transforms[ original_gbk_transform] input_pcoll = only_element(transform_proto.inputs.values()) output_pcoll = only_element(transform_proto.outputs.values()) pre_gbk_coder = coders[safe_coders[ pipeline_components.pcollections[input_pcoll].coder_id]] post_gbk_coder = coders[safe_coders[ pipeline_components.pcollections[output_pcoll].coder_id]] pcoll_buffers[pcoll_id] = _GroupingBuffer( pre_gbk_coder, post_gbk_coder) pcoll_buffers[pcoll_id].append(output.data) else: # These should be the only two identifiers we produce for now, # but special side input writes may go here. raise NotImplementedError(pcoll_id)