[ 
https://issues.apache.org/jira/browse/BEAM-1630?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16275264#comment-16275264
 ] 

ASF GitHub Bot commented on BEAM-1630:
--------------------------------------

chamikaramj commented on a change in pull request #4064: [BEAM-1630] Adds 
support for processing Splittable DoFns using DirectRunner.
URL: https://github.com/apache/beam/pull/4064#discussion_r153898373
 
 

 ##########
 File path: sdks/python/apache_beam/runners/direct/sdf_direct_runner.py
 ##########
 @@ -0,0 +1,264 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+"""This module contains Splittable DoFn logic that is specific to DirectRunner.
+"""
+
+from threading import Lock
+from threading import Timer
+
+import apache_beam as beam
+from apache_beam import TimeDomain
+from apache_beam import pvalue
+from apache_beam.io.iobase import RestrictionTracker
+from apache_beam.pipeline import PTransformOverride
+from apache_beam.runners.common import DoFnContext
+from apache_beam.runners.common import DoFnInvoker
+from apache_beam.runners.common import DoFnSignature
+from apache_beam.runners.common import OutputProcessor
+from apache_beam.runners.direct.evaluation_context import DirectStepContext
+from apache_beam.runners.direct.util import KeyedWorkItem
+from apache_beam.runners.direct.watermark_manager import WatermarkManager
+from apache_beam.runners.sdf_common import ElementAndRestriction
+from apache_beam.runners.sdf_common import ProcessKeyedElements
+from apache_beam.transforms.core import ProcessContinuation
+from apache_beam.transforms.ptransform import PTransform
+from apache_beam.transforms.trigger import _ValueStateTag
+from apache_beam.utils.windowed_value import WindowedValue
+
+
+class ProcessKeyedElementsViaKeyedWorkItemsOverride(PTransformOverride):
+  """A transform override for ProcessElements transform."""
+
+  def get_matcher(self):
+    def _matcher(applied_ptransform):
+      return isinstance(
+          applied_ptransform.transform, ProcessKeyedElements)
+
+    return _matcher
+
+  def get_replacement_transform(self, ptransform):
+    return ProcessKeyedElementsViaKeyedWorkItems(ptransform)
+
+
+class ProcessKeyedElementsViaKeyedWorkItems(PTransform):
+  """A transform that processes Splittable DoFn input via KeyedWorkItems.
+  """
+
+  def __init__(self, process_keyed_elements_transform):
+    self._process_keyed_elements_transform = process_keyed_elements_transform
+
+  def expand(self, pcoll):
+    return pcoll | beam.core.GroupByKey() | ProcessElements(
+        self._process_keyed_elements_transform)
+
+
+class ProcessElements(PTransform):
+  """Processes keyed input via Splittable DoFn objects."""
 
 Review comment:
   Updated.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Add Splittable DoFn to Python SDK
> ---------------------------------
>
>                 Key: BEAM-1630
>                 URL: https://issues.apache.org/jira/browse/BEAM-1630
>             Project: Beam
>          Issue Type: Improvement
>          Components: sdk-py-core
>            Reporter: Chamikara Jayalath
>            Assignee: Chamikara Jayalath
>
> Splittable DoFn [1] is currently being implemented for Java SDK [2]. We 
> should add this to Python SDK as well.
> Following document proposes an API for this.
> https://docs.google.com/document/d/1h_zprJrOilivK2xfvl4L42vaX4DMYGfH1YDmi-s_ozM/edit?usp=sharing
> [1] https://s.apache.org/splittable-do-fn
> [2] 
> https://lists.apache.org/thread.html/0ce61ac162460a149d5c93cdface37cc383f8030fe86ca09e5699b18@%3Cdev.beam.apache.org%3E



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to