[ 
https://issues.apache.org/jira/browse/BEAM-3884?focusedWorklogId=113822&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-113822
 ]

ASF GitHub Bot logged work on BEAM-3884:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 20/Jun/18 17:13
            Start Date: 20/Jun/18 17:13
    Worklog Time Spent: 10m 
      Work Description: robertwb closed pull request #5685: [BEAM-3884] Use 
Impulse primitive for Read.
URL: https://github.com/apache/beam/pull/5685
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/sdks/python/apache_beam/io/iobase.py 
b/sdks/python/apache_beam/io/iobase.py
index 612bcbaf54e..7f9750b9609 100644
--- a/sdks/python/apache_beam/io/iobase.py
+++ b/sdks/python/apache_beam/io/iobase.py
@@ -839,9 +839,27 @@ def __init__(self, source):
     self.source = source
 
   def expand(self, pbegin):
+    from apache_beam.options.pipeline_options import DebugOptions
+    from apache_beam.transforms import util
+
     assert isinstance(pbegin, pvalue.PBegin)
     self.pipeline = pbegin.pipeline
-    return pvalue.PCollection(self.pipeline)
+
+    debug_options = self.pipeline._options.view_as(DebugOptions)
+    if debug_options.experiments and 'beam_fn_api' in 
debug_options.experiments:
+      NUM_SPLITS = 1000
+      source = self.source
+      return (
+          pbegin
+          | core.Impulse()
+          | 'Split' >> core.FlatMap(lambda _: source.split(NUM_SPLITS))
+          | util.Reshuffle()
+          | 'ReadSplits' >> core.FlatMap(lambda split: split.source.read(
+              split.source.get_range_tracker(
+                  split.start_position, split.stop_position))))
+    else:
+      # Treat Read itself as a primitive.
+      return pvalue.PCollection(self.pipeline)
 
   def get_windowing(self, unused_inputs):
     return core.Windowing(window.GlobalWindows())
diff --git a/sdks/python/apache_beam/transforms/core.py 
b/sdks/python/apache_beam/transforms/core.py
index 94d5bb9c960..03ae6870efd 100644
--- a/sdks/python/apache_beam/transforms/core.py
+++ b/sdks/python/apache_beam/transforms/core.py
@@ -76,6 +76,7 @@
     'WindowInto',
     'Flatten',
     'Create',
+    'Impulse',
     ]
 
 
@@ -1894,3 +1895,26 @@ def _create_source(serialized_values, coder):
     from apache_beam.transforms.create_source import _CreateSource
 
     return _CreateSource(serialized_values, coder)
+
+
+class Impulse(PTransform):
+  """Impulse primitive."""
+
+  def expand(self, pbegin):
+    if not isinstance(pbegin, pvalue.PBegin):
+      raise TypeError(
+          'Input to Impulse transform must be a PBegin but found %s' % pbegin)
+    return pvalue.PCollection(pbegin.pipeline)
+
+  def get_windowing(self, inputs):
+    return Windowing(GlobalWindows())
+
+  def infer_output_type(self, unused_input_type):
+    return bytes
+
+  def to_runner_api_parameter(self, unused_context):
+    return common_urns.primitives.IMPULSE.urn, None
+
+  @PTransform.register_urn(common_urns.primitives.IMPULSE.urn, None)
+  def from_runner_api_parameter(unused_parameter, unused_context):
+    return Impulse()


 

----------------------------------------------------------------
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


Issue Time Tracking
-------------------

    Worklog Id:     (was: 113822)
    Time Spent: 1h 50m  (was: 1h 40m)

> Python SDK supports Impulse as a primitive transform
> ----------------------------------------------------
>
>                 Key: BEAM-3884
>                 URL: https://issues.apache.org/jira/browse/BEAM-3884
>             Project: Beam
>          Issue Type: Sub-task
>          Components: sdk-py-core
>            Reporter: Ben Sidhom
>            Assignee: Robert Bradshaw
>            Priority: Major
>          Time Spent: 1h 50m
>  Remaining Estimate: 0h
>
> Portable runners require Impulse to be the only root nodes of pipelines. The 
> Python SDK should provide this for pipeline construction.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to