[
https://issues.apache.org/jira/browse/BEAM-1630?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16275284#comment-16275284
]
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_r154452810
##########
File path: sdks/python/apache_beam/runners/sdf_common.py
##########
@@ -0,0 +1,161 @@
+#
+# 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's common to all runners."""
+
+import uuid
+
+import apache_beam as beam
+from apache_beam import pvalue
+from apache_beam.coders import typecoders
+from apache_beam.pipeline import AppliedPTransform
+from apache_beam.pipeline import PTransformOverride
+from apache_beam.runners.common import DoFnInvoker
+from apache_beam.runners.common import DoFnSignature
+from apache_beam.transforms.core import ParDo
+from apache_beam.transforms.ptransform import PTransform
+
+
+class SplittableParDoOverride(PTransformOverride):
+ """A transform override for ParDo transformss of SplittableDoFns.
+
+ Replaces the ParDo transform with a SplittableParDo transform that performs
+ SDF specific logic.
+ """
+
+ def get_matcher(self):
+ def _matcher(applied_ptransform):
+ assert isinstance(applied_ptransform, AppliedPTransform)
+ transform = applied_ptransform.transform
+ if isinstance(transform, ParDo):
+ signature = DoFnSignature(transform.fn)
+ return signature.is_splittable_dofn()
+
+ return _matcher
+
+ def get_replacement_transform(self, ptransform):
+ assert isinstance(ptransform, ParDo)
+ do_fn = ptransform.fn
+ signature = DoFnSignature(do_fn)
+ if signature.is_splittable_dofn():
+ return SplittableParDo(ptransform)
+ else:
+ return ptransform
+
+
+class SplittableParDo(PTransform):
+ """A transform that processes a PCollection using a Splittable DoFn."""
+
+ def __init__(self, ptransform):
+ assert isinstance(ptransform, ParDo)
+ self._ptransform = ptransform
+
+ def expand(self, pcoll):
+ sdf = self._ptransform.fn
+ signature = DoFnSignature(sdf)
+ invoker = DoFnInvoker.create_invoker(signature, process_invocation=False)
+
+ element_coder = typecoders.registry.get_coder(pcoll.element_type)
+ restriction_coder = invoker.invoke_restriction_coder()
+
+ keyed_elements = (pcoll
+ | 'pair' >> ParDo(PairWithRestrictionFn(sdf))
+ | 'split' >> ParDo(SplitRestrictionFn(sdf))
+ | 'explode' >> ParDo(ExplodeWindowsFn())
+ | 'random' >> ParDo(RandomUniqueKeyFn()))
+
+ return keyed_elements | ProcessKeyedElements(
+ sdf, element_coder, restriction_coder,
+ pcoll.windowing, self._ptransform.args, self._ptransform.kwargs)
+
+
+class ElementAndRestriction(object):
+ """A holder for an element and a restriction."""
+
+ def __init__(self, element, restriction):
+ self.element = element
+ self.restriction = restriction
+
+
+class PairWithRestrictionFn(beam.DoFn):
+ """A transform that pairs each element with a restriction."""
+
+ def __init__(self, do_fn):
+ signature = DoFnSignature(do_fn)
+ self._invoker = DoFnInvoker.create_invoker(
+ signature, process_invocation=False)
+
+ def process(self, element, window=beam.DoFn.WindowParam, *args, **kwargs):
+ initial_restriction = self._invoker.invoke_initial_restriction(element)
+ yield ElementAndRestriction(element, initial_restriction)
+
+
+class SplitRestrictionFn(beam.DoFn):
+ """A transform that perform initial splitting of Splittable DoFn inputs."""
+
+ def __init__(self, do_fn):
+ signature = DoFnSignature(do_fn)
+ self._invoker = DoFnInvoker.create_invoker(
+ signature, process_invocation=False)
+
+ def process(self, element_and_restriction, *args, **kwargs):
+ element = element_and_restriction.element
+ restriction = element_and_restriction.restriction
+ restriction_parts = self._invoker.invoke_split(
+ element,
+ restriction)
+ for part in restriction_parts:
+ yield ElementAndRestriction(element, part)
+
+
+class ExplodeWindowsFn(beam.DoFn):
+ """A transform that forces the runner to explode windows.
+
+ This is done to make sure that Splittable DoFn proceses an element for each
of
+ the windows that element belongs to.
+ """
+
+ def process(self, element, window=beam.DoFn.WindowParam, *args, **kwargs):
+ yield element
+
+
+class RandomUniqueKeyFn(beam.DoFn):
+ """A transform that assigns a unique key to each element."""
+
+ def process(self, element, window=beam.DoFn.WindowParam, *args, **kwargs):
+ yield (uuid.uuid4().bytes, element)
Review comment:
Added TODOs to here and iobase._WriteBundleDoFn. I think collisions are
extremely rare for uuid.uuid4() though. Also, added an assertion to force a
failure here if a collision is detected.
----------------------------------------------------------------
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:
[email protected]
> 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)