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

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

                Author: ASF GitHub Bot
            Created on: 18/Sep/18 07:43
            Start Date: 18/Sep/18 07:43
    Worklog Time Spent: 10m 
      Work Description: robertwb closed pull request #6405: [BEAM-5395] Chunk 
data streams.
URL: https://github.com/apache/beam/pull/6405
 
 
   

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/coders/slow_stream.py 
b/sdks/python/apache_beam/coders/slow_stream.py
index 4bdece6072b..3b740c700ce 100644
--- a/sdks/python/apache_beam/coders/slow_stream.py
+++ b/sdks/python/apache_beam/coders/slow_stream.py
@@ -34,15 +34,18 @@ class OutputStream(object):
 
   def __init__(self):
     self.data = []
+    self.byte_count = 0
 
   def write(self, b, nested=False):
     assert isinstance(b, bytes)
     if nested:
       self.write_var_int64(len(b))
     self.data.append(b)
+    self.byte_count += len(b)
 
   def write_byte(self, val):
     self.data.append(chr(val).encode('latin-1'))
+    self.byte_count += 1
 
   def write_var_int64(self, v):
     if v < 0:
@@ -74,7 +77,11 @@ def get(self):
     return b''.join(self.data)
 
   def size(self):
-    return len(self.data)
+    return self.byte_count
+
+  def _clear(self):
+    self.data = []
+    self.byte_count = 0
 
 
 class ByteCountingOutputStream(OutputStream):
diff --git a/sdks/python/apache_beam/coders/stream.pxd 
b/sdks/python/apache_beam/coders/stream.pxd
index ade9b722c6e..faee45aac99 100644
--- a/sdks/python/apache_beam/coders/stream.pxd
+++ b/sdks/python/apache_beam/coders/stream.pxd
@@ -34,6 +34,7 @@ cdef class OutputStream(object):
   cpdef bytes get(self)
   cpdef size_t size(self) except? -1
   cdef extend(self, size_t missing)
+  cpdef _clear(self)
 
 
 cdef class ByteCountingOutputStream(OutputStream):
diff --git a/sdks/python/apache_beam/coders/stream.pyx 
b/sdks/python/apache_beam/coders/stream.pyx
index 7c9521a8637..7e1e07bfcab 100644
--- a/sdks/python/apache_beam/coders/stream.pyx
+++ b/sdks/python/apache_beam/coders/stream.pyx
@@ -108,6 +108,9 @@ cdef class OutputStream(object):
     self.data = <char*>libc.stdlib.realloc(self.data, self.buffer_size)
     assert self.data, "OutputStream realloc failed."
 
+  cpdef _clear(self):
+    self.pos = 0
+
 
 cdef class ByteCountingOutputStream(OutputStream):
   """An output string stream implementation that only counts the bytes.
diff --git a/sdks/python/apache_beam/runners/portability/fn_api_runner.py 
b/sdks/python/apache_beam/runners/portability/fn_api_runner.py
index 593c6e3da21..9e39ca82c84 100644
--- a/sdks/python/apache_beam/runners/portability/fn_api_runner.py
+++ b/sdks/python/apache_beam/runners/portability/fn_api_runner.py
@@ -1083,11 +1083,19 @@ def __init__(self, sdk_harness_factory=None):
           futures.ThreadPoolExecutor(max_workers=10))
       self.control_port = self.control_server.add_insecure_port('[::]:0')
 
-      self.data_server = 
grpc.server(futures.ThreadPoolExecutor(max_workers=10))
+      # Options to have no limits (-1) on the size of the messages
+      # received or sent over the data plane. The actual buffer size
+      # is controlled in a layer above.
+      no_max_message_sizes = [("grpc.max_receive_message_length", -1),
+                              ("grpc.max_send_message_length", -1)]
+      self.data_server = grpc.server(
+          futures.ThreadPoolExecutor(max_workers=10),
+          options=no_max_message_sizes)
       self.data_port = self.data_server.add_insecure_port('[::]:0')
 
       self.state_server = grpc.server(
-          futures.ThreadPoolExecutor(max_workers=10))
+          futures.ThreadPoolExecutor(max_workers=10),
+          options=no_max_message_sizes)
       self.state_port = self.state_server.add_insecure_port('[::]:0')
 
       self.control_handler = BeamFnControlServicer()
diff --git a/sdks/python/apache_beam/runners/portability/fn_api_runner_test.py 
b/sdks/python/apache_beam/runners/portability/fn_api_runner_test.py
index 277a817dd2a..1064f626d66 100644
--- a/sdks/python/apache_beam/runners/portability/fn_api_runner_test.py
+++ b/sdks/python/apache_beam/runners/portability/fn_api_runner_test.py
@@ -31,6 +31,7 @@
 from apache_beam.metrics.execution import MetricsEnvironment
 from apache_beam.metrics.metricbase import MetricName
 from apache_beam.runners.portability import fn_api_runner
+from apache_beam.runners.worker import data_plane
 from apache_beam.runners.worker import sdk_worker
 from apache_beam.runners.worker import statesampler
 from apache_beam.testing.util import assert_that
@@ -272,6 +273,25 @@ def test_windowing(self):
              | beam.Map(lambda k_vs1: (k_vs1[0], sorted(k_vs1[1]))))
       assert_that(res, equal_to([('k', [1, 2]), ('k', [100, 101, 102])]))
 
+  def test_large_elements(self):
+    with self.create_pipeline() as p:
+      big = (p
+             | beam.Create(['a', 'a', 'b'])
+             | beam.Map(lambda x: (x, x * 
data_plane._DEFAULT_FLUSH_THRESHOLD)))
+
+      side_input_res = (
+          big
+          | beam.Map(lambda x, side: (x[0], side.count(x[0])),
+                     beam.pvalue.AsList(big | beam.Map(lambda x: x[0]))))
+      assert_that(side_input_res,
+                  equal_to([('a', 2), ('a', 2), ('b', 1)]), label='side')
+
+      gbk_res = (
+          big
+          | beam.GroupByKey()
+          | beam.Map(lambda x: x[0]))
+      assert_that(gbk_res, equal_to(['a', 'b']), label='gbk')
+
   def test_error_message_includes_stage(self):
     with self.assertRaises(BaseException) as e_cm:
       with self.create_pipeline() as p:
diff --git a/sdks/python/apache_beam/runners/worker/bundle_processor.py 
b/sdks/python/apache_beam/runners/worker/bundle_processor.py
index 33c8744a7b4..4b7e9cda1bf 100644
--- a/sdks/python/apache_beam/runners/worker/bundle_processor.py
+++ b/sdks/python/apache_beam/runners/worker/bundle_processor.py
@@ -88,6 +88,7 @@ def set_output_stream(self, output_stream):
   def process(self, windowed_value):
     self.windowed_coder_impl.encode_to_stream(
         windowed_value, self.output_stream, True)
+    self.output_stream.maybe_flush()
 
   def finish(self):
     self.output_stream.close()
diff --git a/sdks/python/apache_beam/runners/worker/data_plane.py 
b/sdks/python/apache_beam/runners/worker/data_plane.py
index bb3cc2a85bb..794ed4d13a2 100644
--- a/sdks/python/apache_beam/runners/worker/data_plane.py
+++ b/sdks/python/apache_beam/runners/worker/data_plane.py
@@ -45,12 +45,26 @@
 # This module is experimental. No backwards-compatibility guarantees.
 
 
+_DEFAULT_FLUSH_THRESHOLD = 10 << 20  # 10MB
+
+
 class ClosableOutputStream(type(coder_impl.create_OutputStream())):
   """A Outputstream for use with CoderImpls that has a close() method."""
 
-  def __init__(self, close_callback=None):
+  def __init__(self,
+               close_callback=None,
+               flush_callback=None,
+               flush_threshold=_DEFAULT_FLUSH_THRESHOLD):
     super(ClosableOutputStream, self).__init__()
     self._close_callback = close_callback
+    self._flush_callback = flush_callback
+    self._flush_threshold = flush_threshold
+
+  # This must be called explicitly to avoid flushing partial elements.
+  def maybe_flush(self):
+    if self._flush_callback and self.size() > self._flush_threshold:
+      self._flush_callback(self.get())
+      self._clear()
 
   def close(self):
     if self._close_callback:
@@ -137,7 +151,8 @@ def add_to_inverse_output(data):
               instruction_reference=instruction_id,
               target=target,
               data=data))
-    return ClosableOutputStream(add_to_inverse_output)
+    return ClosableOutputStream(
+        add_to_inverse_output, flush_callback=add_to_inverse_output)
 
   def close(self):
     pass
@@ -202,10 +217,6 @@ def input_elements(self, instruction_id, expected_targets):
       self._clean_receiving_queue(instruction_id)
 
   def output_stream(self, instruction_id, target):
-    # TODO: Return an output stream that sends data
-    # to the Runner once a fixed size buffer is full.
-    # Currently we buffer all the data before sending
-    # any messages.
     def add_to_send_queue(data):
       if data:
         self._to_send.put(
@@ -213,13 +224,17 @@ def add_to_send_queue(data):
                 instruction_reference=instruction_id,
                 target=target,
                 data=data))
+
+    def close_callback(data):
+      add_to_send_queue(data)
       # End of stream marker.
       self._to_send.put(
           beam_fn_api_pb2.Elements.Data(
               instruction_reference=instruction_id,
               target=target,
               data=''))
-    return ClosableOutputStream(add_to_send_queue)
+    return ClosableOutputStream(
+        close_callback, flush_callback=add_to_send_queue)
 
   def _write_outputs(self):
     done = False


 

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


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

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

> BeamPython data plane streams data
> ----------------------------------
>
>                 Key: BEAM-5395
>                 URL: https://issues.apache.org/jira/browse/BEAM-5395
>             Project: Beam
>          Issue Type: Bug
>          Components: sdk-py-harness
>            Reporter: Robert Bradshaw
>            Assignee: Robert Bradshaw
>            Priority: Major
>          Time Spent: 1h
>  Remaining Estimate: 0h
>
> Currently the default implementation is to buffer all data for the bundle. 
> Experiments were made splitting at arbitrary byte boundaries, but it appears 
> that Java requires messages to be split on element boundaries. For now we 
> should implement that in Python (even if this means not being able to split 
> up large elements among multiple messages). 



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

Reply via email to