def get_windowing(self, unused_inputs): return core.Windowing(window.GlobalWindows())
def get_windowing(self, _): return core.Windowing(window.GlobalWindows())
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)