robertwb commented on a change in pull request #12995:
URL: https://github.com/apache/beam/pull/12995#discussion_r572389255



##########
File path: 
sdks/python/apache_beam/runners/portability/fn_api_runner/execution.py
##########
@@ -298,6 +301,198 @@ def from_runner_api_parameter(window_coder_id, context):
         context.coders[window_coder_id.decode('utf-8')])
 
 
+class GenericMergingWindowFn(window.WindowFn):
+
+  URN = 'internal-generic-merging'
+
+  TO_SDK_TRANSFORM = 'read'
+  FROM_SDK_TRANSFORM = 'write'
+
+  _HANDLES = {}
+
+  def __init__(self, execution_context, windowing_strategy_proto):
+    self._worker_handle = None
+    self._handle_id = handle_id = uuid.uuid4().hex
+    self._HANDLES[handle_id] = self
+    # ExecutionContexts are expensive, we don't want to keep them in the
+    # static dictionary forever.  Instead we hold a weakref and pop self
+    # out of the dict once this context goes away.
+    self._execution_context_ref = weakref.ref(
+        execution_context, lambda _: self._HANDLES.pop(handle_id, None))
+    self._windowing_strategy_proto = windowing_strategy_proto
+    self._process_bundle_descriptor = None
+    self._counter = 0
+
+  def payload(self):
+    return self._handle_id.encode('utf-8')
+
+  @staticmethod
+  @window.urns.RunnerApiFn.register_urn(URN, bytes)
+  def from_runner_api_parameter(handle_id, unused_context):
+    return GenericMergingWindowFn._HANDLES[handle_id.decode('utf-8')]
+
+  def assign(self, assign_context):
+    raise NotImplementedError()
+
+  def merge(self, merge_context):
+    worker_handler = self.worker_handle()
+
+    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:
+        (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)
+    # The result was "returned" via the merge callbacks on merge_context above.
+
+  def get_window_coder(self):
+    return self._execution_context_ref().pipeline_context.coders[
+        self._windowing_strategy_proto.window_coder_id]
+
+  def worker_handle(self):
+    if self._worker_handle is None:
+      worker_handler_manager = self._execution_context_ref(
+      ).worker_handler_manager
+      self._worker_handler = worker_handler_manager.get_worker_handlers(

Review comment:
       Good catch. Fixed.

##########
File path: sdks/python/apache_beam/runners/portability/flink_runner_test.py
##########
@@ -400,6 +400,9 @@ def test_callbacks_with_exception(self):
   def test_register_finalizations(self):
     raise unittest.SkipTest("BEAM-6868")
 
+  def test_custom_merging_window(self):

Review comment:
       Done.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to