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