You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by ro...@apache.org on 2016/09/23 20:54:33 UTC

[4/6] incubator-beam git commit: Implement liquid sharding for concat source.

Implement liquid sharding for concat source.


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/c7d5a37d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/c7d5a37d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/c7d5a37d

Branch: refs/heads/python-sdk
Commit: c7d5a37dce37dacd906202191705054032426ba5
Parents: c1f42e6
Author: Robert Bradshaw <ro...@google.com>
Authored: Tue Sep 13 16:37:06 2016 -0700
Committer: Robert Bradshaw <ro...@google.com>
Committed: Fri Sep 23 13:44:57 2016 -0700

----------------------------------------------------------------------
 sdks/python/apache_beam/io/filebasedsource.py | 166 ++++++++++++++++++---
 sdks/python/apache_beam/io/sources_test.py    |  89 +++++++++++
 2 files changed, 238 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c7d5a37d/sdks/python/apache_beam/io/filebasedsource.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/io/filebasedsource.py b/sdks/python/apache_beam/io/filebasedsource.py
index 5204f04..dd91ef2 100644
--- a/sdks/python/apache_beam/io/filebasedsource.py
+++ b/sdks/python/apache_beam/io/filebasedsource.py
@@ -25,20 +25,146 @@ for more details.
 For an example implementation of ``FileBasedSource`` see ``avroio.AvroSource``.
 """
 
+import bisect
 from multiprocessing.pool import ThreadPool
+import threading
 
 from apache_beam.internal import pickler
 from apache_beam.io import fileio
 from apache_beam.io import iobase
-
-import range_trackers
+from apache_beam.io import range_trackers
 
 MAX_NUM_THREADS_FOR_SIZE_ESTIMATION = 25
 
 
-class _ConcatSource(iobase.BoundedSource):
+class ConcatSource(iobase.BoundedSource):
   """A ``BoundedSource`` that can group a set of ``BoundedSources``."""
 
+  class ConcatRangeTracker(iobase.RangeTracker):
+
+    def __init__(self, start, end, sources, weights=None):
+      super(ConcatSource.ConcatRangeTracker, self).__init__()
+      self._start = start
+      self._end = end
+      self._lock = threading.RLock()
+      self._sources = sources
+      self._range_trackers = [None] * len(self._sources)
+      self._claimed_source_ix = self._start[0]
+
+      if weights is None:
+        n = max(1, end[0] - start[0])
+        self._cumulative_weights = [
+          max(0, min(1, float(k) / n))
+          for k in range(-start[0], len(self._sources) - start[0] + 1)]
+      else:
+        assert len(sources) == len(weights)
+        relevant_weights = weights[start[0]:end[0] + 1]
+        # TODO(robertwb): Implement fraction-at-position to properly scale
+        # partial start and end sources.
+        total = sum(relevant_weights)
+        running_total = [0]
+        for w in relevant_weights:
+          running_total.append(max(1, running_total[-1] + w / total))
+        running_total[-1] = 1  # In case of rounding error.
+        self._cumulative_weights = (
+          [0] * start[0]
+          + running_total
+          + [1] * (len(self._sources) - end[0]))
+
+    def start_position(self):
+      return self._start
+
+    def stop_position(self):
+      return self._end
+
+    def try_claim(self, pos):
+      source_ix, source_pos = pos
+      with self._lock:
+        if source_ix > self._end[0]:
+          return False
+        elif source_ix == self._end[0] and self._end[1] is None:
+          return False
+        else:
+          self._claimed_source_ix = source_ix
+          if source_pos is None:
+            return True
+          else:
+            return self.sub_range_tracker(source_ix).try_claim(source_pos)
+
+    def try_split(self, pos):
+      source_ix, source_pos = pos
+      with self._lock:
+        if source_ix < self._claimed_source_ix:
+          # Already claimed.
+          return None
+        elif source_ix > self._end[0]:
+          # After end.
+          return None
+        elif source_ix == self._end[0] and self._end[1] is None:
+          # At/after end.
+          return None
+        else:
+          if source_ix > self._claimed_source_ix:
+            # Prefer to split on even boundary.
+            split_pos = None
+            ratio = self._cumulative_weights[source_ix]
+          else:
+            # Split the current subsource.
+            split = self.sub_range_tracker(source_ix).try_split(
+                source_pos)
+            if not split:
+              return None
+            split_pos, frac = split
+            ratio = self.local_to_global(source_ix, frac)
+
+          self._end = source_ix, split_pos
+          self._cumulative_weights = [min(w / ratio, 1)
+                                        for w in self._cumulative_weights]
+          return (source_ix, split_pos), ratio
+
+    def set_current_position(self, pos):
+      raise NotImplementedError('Should only be called on sub-trackers')
+
+    def position_at_fraction(self, fraction):
+      source_ix, source_frac = self.global_to_local(fraction)
+      if source_ix == len(self._sources):
+        return (source_ix, None)
+      else:
+        return (source_ix,
+                self.sub_range_tracker(source_ix).position_at_fraction(
+                    source_frac))
+
+    def fraction_consumed(self):
+      with self._lock:
+        return self.local_to_global(self._claimed_source_ix,
+                                    self.sub_range_tracker(
+                                        self._claimed_source_ix)
+                                        .fraction_consumed())
+
+    def local_to_global(self, source_ix, source_frac):
+      cw = self._cumulative_weights
+      return cw[source_ix] + source_frac * (cw[source_ix + 1] - cw[source_ix])
+
+    def global_to_local(self, frac):
+      if frac == 1:
+        return (len(self._sources), 0)
+      else:
+        cw = self._cumulative_weights
+        source_ix = bisect.bisect(cw, frac) - 1
+        return (source_ix,
+                (frac - cw[source_ix]) / (cw[source_ix + 1] - cw[source_ix]))
+
+    def sub_range_tracker(self, source_ix):
+      assert self._start[0] <= source_ix <= self._end[0]
+      if self._range_trackers[source_ix] is None:
+        with self._lock:
+          if self._range_trackers[source_ix] is None:
+            self._range_trackers[source_ix] = (
+                self._sources[source_ix].get_range_tracker(
+                    self._start[1] if source_ix == self._start[0] else None,
+                    self._end[1] if source_ix == self._end[0] else None))
+      return self._range_trackers[source_ix]
+
   def __init__(self, sources):
     self._sources = sources
 
@@ -64,20 +190,23 @@ class _ConcatSource(iobase.BoundedSource):
       for bundle in source.split(desired_bundle_size, None, None):
         yield bundle
 
-  def get_range_tracker(self, start_position, stop_position):
-    assert start_position is None
-    assert stop_position is None
-    # This will be invoked only when FileBasedSource is read without splitting.
-    # For that case, we only support reading the whole source.
-    return range_trackers.OffsetRangeTracker(0, len(self.sources))
+  def get_range_tracker(self, start_position=None, stop_position=None):
+    if start_position is None:
+      start_position = (0, None)
+    if stop_position is None:
+      stop_position = (len(self._sources), None)
+    return self.ConcatRangeTracker(start_position, stop_position, self._sources)
 
   def read(self, range_tracker):
-    for index, sub_source in enumerate(self.sources):
-      if not range_tracker.try_claim(index):
-        return
-
-      sub_source_tracker = sub_source.get_range_tracker(None, None)
-      for record in sub_source.read(sub_source_tracker):
+    start_source, _ = range_tracker.start_position()
+    stop_source, stop_pos = range_tracker.stop_position()
+    if stop_pos is not None:
+      stop_source += 1
+    for source_ix in range(start_source, stop_source):
+      if not range_tracker.try_claim((source_ix, None)):
+        break
+      for record in self._sources[source_ix].read(
+          range_tracker.sub_range_tracker(source_ix)):
         yield record
 
   def default_output_coder(self):
@@ -87,7 +216,10 @@ class _ConcatSource(iobase.BoundedSource):
       return self._sources[0].default_output_coder()
     else:
       # Defaulting to PickleCoder.
-      return super(_ConcatSource, self).default_output_coder()
+      return super(ConcatSource, self).default_output_coder()
+
+
+_ConcatSource = ConcatSource
 
 
 class FileBasedSource(iobase.BoundedSource):
@@ -162,7 +294,7 @@ class FileBasedSource(iobase.BoundedSource):
             sizes[index],
             min_bundle_size=self._min_bundle_size)
         single_file_sources.append(single_file_source)
-      self._concat_source = _ConcatSource(single_file_sources)
+      self._concat_source = ConcatSource(single_file_sources)
     return self._concat_source
 
   def open_file(self, file_name):

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c7d5a37d/sdks/python/apache_beam/io/sources_test.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/io/sources_test.py b/sdks/python/apache_beam/io/sources_test.py
index 293f437..3f83d80 100644
--- a/sdks/python/apache_beam/io/sources_test.py
+++ b/sdks/python/apache_beam/io/sources_test.py
@@ -26,7 +26,9 @@ import apache_beam as beam
 
 from apache_beam import coders
 from apache_beam.io import iobase
+from apache_beam.io.filebasedsource import ConcatSource
 from apache_beam.io import range_trackers
+from apache_beam.io import source_test_utils
 from apache_beam.transforms.util import assert_that
 from apache_beam.transforms.util import equal_to
 
@@ -80,6 +82,45 @@ class LineSource(iobase.BoundedSource):
     return coders.BytesCoder()
 
 
+class RangeSource(iobase.BoundedSource):
+
+  def __init__(self, start, end, split_freq=1):
+    assert start <= end
+    self._start = start
+    self._end = end
+    self._split_freq = split_freq
+
+  def _normalize(self, start_position, end_position):
+    return (self._start if start_position is None else start_position,
+            self._end if end_position is None else end_position)
+
+  def _round_up(self, index):
+    """Rounds up to the nearest mulitple of split_freq."""
+    return index - index % -self._split_freq
+
+  def estimate_size(self):
+    return self._stop - self._start
+
+  def split(self, desired_bundle_size, start_position=None, end_position=None):
+    start, end = self._normalize(start_position, end_position)
+    for sub_start in range(start, end, desired_bundle_size):
+      sub_end = min(self._end, sub_start + desired_bundle_size)
+      yield SourceBundle(RangeSource(sub_start, sub_end, self._split_freq),
+                         sub_end - sub_start)
+
+  def get_range_tracker(self, start_position, end_position):
+    start, end = self._normalize(start_position, end_position)
+    return range_trackers.OffsetRangeTracker(start, end)
+
+  def read(self, range_tracker):
+    for k in range(self._round_up(range_tracker.start_position()),
+                   self._round_up(range_tracker.stop_position())):
+      if k % self._split_freq == 0:
+        if not range_tracker.try_claim(k):
+          return
+      yield k
+
+
 class SourcesTest(unittest.TestCase):
 
   def _create_temp_file(self, contents):
@@ -104,6 +145,54 @@ class SourcesTest(unittest.TestCase):
 
     pipeline.run()
 
+  def test_range_source(self):
+    source_test_utils.assertSplitAtFractionExhaustive(RangeSource(0, 10, 3))
+
+  def test_conact_source(self):
+    source = ConcatSource([RangeSource(0, 4),
+                           RangeSource(4, 8),
+                           RangeSource(8, 12),
+                           RangeSource(12, 16),
+                          ])
+    self.assertEqual(list(source.read(source.get_range_tracker())),
+                     range(16))
+    self.assertEqual(list(source.read(source.get_range_tracker((1, None),
+                                                               (2, 10)))),
+                     range(4, 10))
+    range_tracker = source.get_range_tracker(None, None)
+    self.assertEqual(range_tracker.position_at_fraction(0), (0, 0))
+    self.assertEqual(range_tracker.position_at_fraction(.5), (2, 8))
+    self.assertEqual(range_tracker.position_at_fraction(.625), (2, 10))
+
+    # Simulate a read.
+    self.assertEqual(range_tracker.try_claim((0, None)), True)
+    self.assertEqual(range_tracker.sub_range_tracker(0).try_claim(2), True)
+    self.assertEqual(range_tracker.fraction_consumed(), 0.125)
+
+    self.assertEqual(range_tracker.try_claim((1, None)), True)
+    self.assertEqual(range_tracker.sub_range_tracker(1).try_claim(6), True)
+    self.assertEqual(range_tracker.fraction_consumed(), 0.375)
+    self.assertEqual(range_tracker.try_split((0, 1)), None)
+    self.assertEqual(range_tracker.try_split((1, 5)), None)
+
+    self.assertEqual(range_tracker.try_split((3, 14)), ((3, None), 0.75))
+    self.assertEqual(range_tracker.try_claim((3, None)), False)
+    self.assertEqual(range_tracker.sub_range_tracker(1).try_claim(7), True)
+    self.assertEqual(range_tracker.try_claim((2, None)), True)
+    self.assertEqual(range_tracker.sub_range_tracker(2).try_claim(9), True)
+
+    self.assertEqual(range_tracker.try_split((2, 8)), None)
+    self.assertEqual(range_tracker.try_split((2, 11)), ((2, 11), 11. / 12))
+    self.assertEqual(range_tracker.sub_range_tracker(2).try_claim(10), True)
+    self.assertEqual(range_tracker.sub_range_tracker(2).try_claim(11), False)
+
+  def test_conact_source_exhaustive(self):
+    source = ConcatSource([RangeSource(0, 10),
+                           RangeSource(100, 110),
+                           RangeSource(1000, 1010),
+                          ])
+    source_test_utils.assertSplitAtFractionExhaustive(source)
+
 if __name__ == '__main__':
   logging.getLogger().setLevel(logging.INFO)
   unittest.main()