[
https://issues.apache.org/jira/browse/BEAM-1630?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16275251#comment-16275251
]
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_r152440266
##########
File path: sdks/python/apache_beam/runners/direct/transform_evaluator.py
##########
@@ -826,3 +830,74 @@ def finish_bundle(self):
None, '', TimeDomain.WATERMARK, WatermarkManager.WATERMARK_POS_INF)
return TransformResult(self, [], [], None, {None: hold})
+
+
+class _ProcessElemenetsEvaluator(_TransformEvaluator):
+ """An evaluator for sdf_direct_runner.ProcessElements transform."""
+
+ DEFAULT_MAX_NUM_OUTPUTS = 100
+ DEFAULT_MAX_DURATION = 1
+
+ def __init__(self, evaluation_context, applied_ptransform,
+ input_committed_bundle, side_inputs, scoped_metrics_container):
+ super(_ProcessElemenetsEvaluator, self).__init__(
+ evaluation_context, applied_ptransform, input_committed_bundle,
+ side_inputs, scoped_metrics_container)
+
+ process_elements_transform = applied_ptransform.transform
+ assert isinstance(process_elements_transform, ProcessElements)
+
+ # Replacing the do_fn of the transform with a wrapper do_fn that performs
+ # SDF magic.
+ transform = applied_ptransform.transform
+ sdf = transform.sdf
+ self._process_fn = transform.new_process_fn(sdf)
+ transform.dofn = self._process_fn
+
+ assert isinstance(self._process_fn, ProcessFn)
+
+ self.step_context = self._execution_context.get_step_context()
+ # self.global_state = self.step_context.get_keyed_state(None)
+ self._process_fn.set_step_context(self.step_context)
+
+ process_element_invoker = (
+ SDFProcessElementInvoker(
+ max_num_outputs=self.DEFAULT_MAX_NUM_OUTPUTS,
+ max_duration=self.DEFAULT_MAX_DURATION))
+ self._process_fn.set_process_element_invoker(process_element_invoker)
+
+ self._par_do_evaluator = _ParDoEvaluator(
+ evaluation_context, applied_ptransform, input_committed_bundle,
+ side_inputs, scoped_metrics_container)
Review comment:
Technically we only have one evaluator here which is
_ProcessElementsEvaluator. _ParDoEvaluator is used as a library. We are simply
using _ParDoEvaluator to evaluate a ParDo where DoFn object is the ProcessFn.
If we decide to duplicate that code this'll involve a significant amount of
code copying (ParDoEvaluator's, start_bundle, process(), finish_bundle()) which
I prefer avoiding.
Also, note that we use a similar implementation for Java SDK:
https://github.com/apache/beam/blob/master/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java#L112
WDYT ?
----------------------------------------------------------------
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)