From 3383c260855150dc9f0182a15f37136ea1b1c93c Mon Sep 17 00:00:00 2001 From: Charles Chen Date: Wed, 24 May 2017 14:50:02 -0700 Subject: [PATCH 01/21] streaming test --- .../apache_beam/runners/direct/executor.py | 14 ++++--- .../runners/direct/streaming_test.py | 41 +++++++++++++++++++ .../runners/direct/transform_evaluator.py | 32 ++++++++++++--- .../runners/direct/transform_result.py | 4 +- .../python/apache_beam/testing/test_stream.py | 5 +++ 5 files changed, 84 insertions(+), 12 deletions(-) create mode 100644 sdks/python/apache_beam/runners/direct/streaming_test.py diff --git a/sdks/python/apache_beam/runners/direct/executor.py b/sdks/python/apache_beam/runners/direct/executor.py index 86db29159f37..b061a352938e 100644 --- a/sdks/python/apache_beam/runners/direct/executor.py +++ b/sdks/python/apache_beam/runners/direct/executor.py @@ -230,13 +230,16 @@ def handle_result(self, input_committed_bundle, transform_result): output_committed_bundles = self._evaluation_context.handle_result( input_committed_bundle, self._timers, transform_result) for output_committed_bundle in output_committed_bundles: - self._all_updates.offer(_ExecutorServiceParallelExecutor._ExecutorUpdate( - output_committed_bundle, None)) + self._all_updates.offer( + _ExecutorServiceParallelExecutor._ExecutorUpdate( + unprocessed_elements=transform_result.unprocessed_elements, + committed_bundle=output_committed_bundle)) return output_committed_bundles def handle_exception(self, exception): self._all_updates.offer( - _ExecutorServiceParallelExecutor._ExecutorUpdate(None, exception)) + _ExecutorServiceParallelExecutor._ExecutorUpdate( + exception=exception)) class TransformExecutor(_ExecutorService.CallableTask): @@ -418,10 +421,11 @@ def offer(self, item): class _ExecutorUpdate(object): """An internal status update on the state of the executor.""" - def __init__(self, produced_bundle=None, exception=None): + def __init__(self, unprocessed_elements=None, committed_bundle=None, exception=None): # Exactly one of them should be not-None assert bool(produced_bundle) != bool(exception) - self.committed_bundle = produced_bundle + self.unprocessed_elements = unprocessed_elements or [] + self.committed_bundle = committed_bundle self.exception = exception self.exc_info = sys.exc_info() if self.exc_info[1] is not exception: diff --git a/sdks/python/apache_beam/runners/direct/streaming_test.py b/sdks/python/apache_beam/runners/direct/streaming_test.py new file mode 100644 index 000000000000..87cc8fdcadc1 --- /dev/null +++ b/sdks/python/apache_beam/runners/direct/streaming_test.py @@ -0,0 +1,41 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import unittest + +import apache_beam as beam +from apache_beam.utils.test_stream import * # TODO +from apache_beam.runners.direct import DirectRunner + + +class StreamingTest(unittest.TestCase): + + def test_test_stream(self): + test_stream = (TestStream() + .advance_watermark_to(10) + .add_elements(['a', 'b', 'c']) + .advance_watermark_to(20) + .add_elements(['d']) + .advance_processing_time(10)) + p = beam.Pipeline(runner=DirectRunner()) + elements = p | test_stream + groups = elements | beam.Map(lambda x: ('k', x)) | beam.GroupByKey() + p.run(test_runner_api=False) + + +if __name__ == '__main__': + unittest.main() diff --git a/sdks/python/apache_beam/runners/direct/transform_evaluator.py b/sdks/python/apache_beam/runners/direct/transform_evaluator.py index b1cb626ca0cb..7f89d0bbb29a 100644 --- a/sdks/python/apache_beam/runners/direct/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/direct/transform_evaluator.py @@ -30,6 +30,7 @@ from apache_beam.runners.direct.watermark_manager import WatermarkManager from apache_beam.runners.direct.transform_result import TransformResult from apache_beam.runners.dataflow.native_io.iobase import _NativeWrite # pylint: disable=protected-access +from apache_beam.testing.test_stream import TestStream from apache_beam.transforms import core from apache_beam.transforms.window import GlobalWindows from apache_beam.transforms.window import WindowedValue @@ -55,6 +56,7 @@ def __init__(self, evaluation_context): core.ParDo: _ParDoEvaluator, core._GroupByKeyOnly: _GroupByKeyOnlyEvaluator, _NativeWrite: _NativeWriteEvaluator, + TestStream: _TestStreamEvaluator, } def for_application( @@ -207,7 +209,25 @@ def _read_values_to_bundles(reader): bundles = _read_values_to_bundles(reader) return TransformResult( - self._applied_ptransform, bundles, None, None, None, None) + self._applied_ptransform, bundles, [], None, None, None, None) + + +class _TestStreamEvaluator(_TransformEvaluator): + """TransformEvaluator for the TestStream transform.""" + + def __init__(self, evaluation_context, applied_ptransform, + input_committed_bundle, side_inputs, scoped_metrics_container): + assert not input_committed_bundle + assert not side_inputs + super(_TestStreamEvaluator, self).__init__( + evaluation_context, applied_ptransform, input_committed_bundle, + side_inputs, scoped_metrics_container) + + def finish_bundle(self): + assert len(self._outputs) == 1 + output_pcollection = list(self._outputs)[0] + + pass class _FlattenEvaluator(_TransformEvaluator): @@ -231,7 +251,7 @@ def process_element(self, element): def finish_bundle(self): bundles = [self.bundle] return TransformResult( - self._applied_ptransform, bundles, None, None, None, None) + self._applied_ptransform, bundles, [], None, None, None, None) class _TaggedReceivers(dict): @@ -320,8 +340,8 @@ def finish_bundle(self): bundles = self._tagged_receivers.values() result_counters = self._counter_factory.get_counters() return TransformResult( - self._applied_ptransform, bundles, None, None, result_counters, None, - self._tagged_receivers.undeclared_in_memory_tag_values) + self._applied_ptransform, bundles, [], None, None, result_counters, + None, self._tagged_receivers.undeclared_in_memory_tag_values) class _GroupByKeyOnlyEvaluator(_TransformEvaluator): @@ -402,7 +422,7 @@ def len_element_fn(element): hold = WatermarkManager.WATERMARK_NEG_INF return TransformResult( - self._applied_ptransform, bundles, state, None, None, hold) + self._applied_ptransform, bundles, [], state, None, None, hold) class _NativeWriteEvaluator(_TransformEvaluator): @@ -459,4 +479,4 @@ def finish_bundle(self): hold = WatermarkManager.WATERMARK_NEG_INF return TransformResult( - self._applied_ptransform, [], state, None, None, hold) + self._applied_ptransform, [], [], state, None, None, hold) diff --git a/sdks/python/apache_beam/runners/direct/transform_result.py b/sdks/python/apache_beam/runners/direct/transform_result.py index febdd202aa0a..97ac4aaf6255 100644 --- a/sdks/python/apache_beam/runners/direct/transform_result.py +++ b/sdks/python/apache_beam/runners/direct/transform_result.py @@ -25,11 +25,13 @@ class TransformResult(object): The result of evaluating an AppliedPTransform with a TransformEvaluator.""" - def __init__(self, applied_ptransform, uncommitted_output_bundles, state, + def __init__(self, applied_ptransform, uncommitted_output_bundles, + unprocessed_elements, state, timer_update, counters, watermark_hold, undeclared_tag_values=None): self.transform = applied_ptransform self.uncommitted_output_bundles = uncommitted_output_bundles + self.unprocessed_elements = unprocessed_elements or [] self.state = state # TODO: timer update is currently unused. self.timer_update = timer_update diff --git a/sdks/python/apache_beam/testing/test_stream.py b/sdks/python/apache_beam/testing/test_stream.py index a06bcd0795f2..5f00f233ac1c 100644 --- a/sdks/python/apache_beam/testing/test_stream.py +++ b/sdks/python/apache_beam/testing/test_stream.py @@ -25,7 +25,9 @@ from apache_beam import coders from apache_beam import pvalue +from apache_beam.transforms import core from apache_beam.transforms import PTransform +from apache_beam.transforms import window from apache_beam.transforms.window import TimestampedValue from apache_beam.utils import timestamp from apache_beam.utils.windowed_value import WindowedValue @@ -104,6 +106,9 @@ def expand(self, pbegin): self.pipeline = pbegin.pipeline return pvalue.PCollection(self.pipeline) + def get_windowing(self, unused_inputs): + return core.Windowing(window.GlobalWindows()) + def _infer_output_coder(self, input_type=None, input_coder=None): return self.coder From 038f3f33297421237814124887564c6f2620711a Mon Sep 17 00:00:00 2001 From: Charles Chen Date: Wed, 24 May 2017 14:53:09 -0700 Subject: [PATCH 02/21] Working TestStream evaluator --- .../runners/direct/bundle_factory.py | 7 +- .../runners/direct/evaluation_context.py | 16 ++- .../apache_beam/runners/direct/executor.py | 104 ++++++++++++----- .../runners/direct/transform_evaluator.py | 106 +++++++++++++++++- .../runners/direct/transform_result.py | 4 +- .../runners/direct/watermark_manager.py | 24 +++- 6 files changed, 217 insertions(+), 44 deletions(-) diff --git a/sdks/python/apache_beam/runners/direct/bundle_factory.py b/sdks/python/apache_beam/runners/direct/bundle_factory.py index ed00b03310bb..a46d10904d95 100644 --- a/sdks/python/apache_beam/runners/direct/bundle_factory.py +++ b/sdks/python/apache_beam/runners/direct/bundle_factory.py @@ -108,13 +108,17 @@ def windowed_values(self): self._initial_windowed_value.windows) def __init__(self, pcollection, stacked=True): - assert isinstance(pcollection, pvalue.PCollection) + # assert isinstance(pcollection, pvalue.PCollection) + print 'INIT PCOLLECTION', pcollection self._pcollection = pcollection self._elements = [] self._stacked = stacked self._committed = False self._tag = None # optional tag information for this bundle + def __repr__(self): + return '' % (self._pcollection, self._elements) + def get_elements_iterable(self, make_copy=False): """Returns iterable elements. @@ -183,6 +187,7 @@ def add(self, element): self._elements.append(element) def output(self, element): + assert isinstance(element, WindowedValue) self.add(element) def commit(self, synchronized_processing_time): diff --git a/sdks/python/apache_beam/runners/direct/evaluation_context.py b/sdks/python/apache_beam/runners/direct/evaluation_context.py index 68d99d373a7e..f992bd09600f 100644 --- a/sdks/python/apache_beam/runners/direct/evaluation_context.py +++ b/sdks/python/apache_beam/runners/direct/evaluation_context.py @@ -199,10 +199,12 @@ def handle_result( the committed bundles contained within the handled result. """ with self._lock: - committed_bundles = self._commit_bundles( - result.uncommitted_output_bundles) + committed_bundles, unprocessed_bundle = self._commit_bundles( + result.uncommitted_output_bundles, + result.unprocessed_bundle) + print 'UNP', unprocessed_bundle self._watermark_manager.update_watermarks( - completed_bundle, result.transform, completed_timers, + completed_bundle, unprocessed_bundle, result.transform, completed_timers, committed_bundles, result.watermark_hold) self._metrics.commit_logical(completed_bundle, @@ -244,14 +246,16 @@ def schedule_pending_unblocked_tasks(self, executor_service): executor_service.submit(task) self._pending_unblocked_tasks = [] - def _commit_bundles(self, uncommitted_bundles): + def _commit_bundles(self, uncommitted_output_bundles, unprocessed_bundle): """Commits bundles and returns a immutable set of committed bundles.""" - for in_progress_bundle in uncommitted_bundles: + for in_progress_bundle in uncommitted_output_bundles: producing_applied_ptransform = in_progress_bundle.pcollection.producer watermarks = self._watermark_manager.get_watermarks( producing_applied_ptransform) in_progress_bundle.commit(watermarks.synchronized_processing_output_time) - return tuple(uncommitted_bundles) + if unprocessed_bundle: + unprocessed_bundle.commit(None) + return tuple(uncommitted_output_bundles), unprocessed_bundle def get_execution_context(self, applied_ptransform): return _ExecutionContext( diff --git a/sdks/python/apache_beam/runners/direct/executor.py b/sdks/python/apache_beam/runners/direct/executor.py index b061a352938e..4462c7fc4758 100644 --- a/sdks/python/apache_beam/runners/direct/executor.py +++ b/sdks/python/apache_beam/runners/direct/executor.py @@ -226,19 +226,26 @@ def __init__(self, evaluation_context, all_updates, timers=None): self._all_updates = all_updates self._timers = timers - def handle_result(self, input_committed_bundle, transform_result): + def handle_result(self, transform_executor, input_committed_bundle, transform_result): + print 'COMPLETIONCALLBACK.handle_result' output_committed_bundles = self._evaluation_context.handle_result( input_committed_bundle, self._timers, transform_result) for output_committed_bundle in output_committed_bundles: self._all_updates.offer( _ExecutorServiceParallelExecutor._ExecutorUpdate( - unprocessed_elements=transform_result.unprocessed_elements, + transform_executor=transform_executor, committed_bundle=output_committed_bundle)) + if transform_result.unprocessed_bundle: + self._all_updates.offer( + _ExecutorServiceParallelExecutor.ExecutorUpdate( + transform_executor=transform_executor, + unprocessed_bundle=transform_result.unprocessed_bundle)) return output_committed_bundles - def handle_exception(self, exception): + def handle_exception(self, transform_executor, exception): self._all_updates.offer( _ExecutorServiceParallelExecutor._ExecutorUpdate( + transform_executor=transform_executor, exception=exception)) @@ -253,12 +260,12 @@ class TransformExecutor(_ExecutorService.CallableTask): """ def __init__(self, transform_evaluator_registry, evaluation_context, - input_bundle, applied_transform, completion_callback, + input_bundle, applied_ptransform, completion_callback, transform_evaluation_state): self._transform_evaluator_registry = transform_evaluator_registry self._evaluation_context = evaluation_context self._input_bundle = input_bundle - self._applied_transform = applied_transform + self._applied_ptransform = applied_ptransform self._completion_callback = completion_callback self._transform_evaluation_state = transform_evaluation_state self._side_input_values = {} @@ -267,11 +274,11 @@ def __init__(self, transform_evaluator_registry, evaluation_context, def call(self): self._call_count += 1 - assert self._call_count <= (1 + len(self._applied_transform.side_inputs)) - metrics_container = MetricsContainer(self._applied_transform.full_label) + assert self._call_count <= (1 + len(self._applied_ptransform.side_inputs)) + metrics_container = MetricsContainer(self._applied_ptransform.full_label) scoped_metrics_container = ScopedMetricsContainer(metrics_container) - for side_input in self._applied_transform.side_inputs: + for side_input in self._applied_ptransform.side_inputs: if side_input not in self._side_input_values: has_result, value = ( self._evaluation_context.get_value_or_schedule_after_output( @@ -283,13 +290,17 @@ def call(self): self._side_input_values[side_input] = value side_input_values = [self._side_input_values[side_input] - for side_input in self._applied_transform.side_inputs] + for side_input in self._applied_ptransform.side_inputs] try: + print 'EVALLLL' evaluator = self._transform_evaluator_registry.for_application( - self._applied_transform, self._input_bundle, + self._applied_ptransform, self._input_bundle, side_input_values, scoped_metrics_container) + # TODO: soon, assert that we have an input bundle, since we get rid of + # the instances where we pass None as the bundle. + if self._input_bundle: for value in self._input_bundle.get_elements_iterable(): evaluator.process_element(value) @@ -301,18 +312,19 @@ def call(self): if self._evaluation_context.has_cache: for uncommitted_bundle in result.uncommitted_output_bundles: self._evaluation_context.append_to_cache( - self._applied_transform, uncommitted_bundle.tag, + self._applied_ptransform, uncommitted_bundle.tag, uncommitted_bundle.get_elements_iterable()) undeclared_tag_values = result.undeclared_tag_values if undeclared_tag_values: for tag, value in undeclared_tag_values.iteritems(): self._evaluation_context.append_to_cache( - self._applied_transform, tag, value) + self._applied_ptransform, tag, value) - self._completion_callback.handle_result(self._input_bundle, result) - return result + self._completion_callback.handle_result(self, self._input_bundle, result) + return result # TODO: not necessary? except Exception as e: # pylint: disable=broad-except - self._completion_callback.handle_exception(e) + logging.warning('Task failed: %s', traceback.format_exc(), exc_info=True) + self._completion_callback.handle_exception(self, e) finally: self._evaluation_context.metrics().commit_physical( self._input_bundle, @@ -356,6 +368,10 @@ def __init__(self, value_to_consumers, transform_evaluator_registry, def start(self, roots): self.root_nodes = frozenset(roots) + self.root_nodes_to_pending_bundles = {} + for root_node in self.root_nodes: + provider = self.transform_evaluator_registry.get_root_bundle_provider(root_node) + self.root_nodes_to_pending_bundles[root_node] = provider.get_root_bundles() self.executor_service.submit( _ExecutorServiceParallelExecutor._MonitorTask(self)) @@ -375,23 +391,30 @@ def schedule_consumers(self, committed_bundle): self.schedule_consumption(applied_ptransform, committed_bundle, self.default_completion_callback) - def schedule_consumption(self, consumer_applied_transform, committed_bundle, + def schedule_unprocessed_bundle(self, applied_ptransform, + unprocessed_bundle): + print 'schedule_unprocessed_bundle', applied_ptransform, unprocessed_bundle + assert applied_ptransform in self.root_nodes + self.root_nodes_to_pending_bundles[applied_ptransform].append(unprocessed_bundle) + + def schedule_consumption(self, consumer_applied_ptransform, committed_bundle, on_complete): """Schedules evaluation of the given bundle with the transform.""" - assert all([consumer_applied_transform, on_complete]) - assert committed_bundle or consumer_applied_transform in self.root_nodes + assert all([consumer_applied_ptransform, on_complete]) + assert committed_bundle or consumer_applied_ptransform in self.root_nodes if (committed_bundle and self.transform_evaluator_registry.should_execute_serially( - consumer_applied_transform)): + consumer_applied_ptransform)): transform_executor_service = self.transform_executor_services.serial( - consumer_applied_transform) + consumer_applied_ptransform) else: transform_executor_service = self.transform_executor_services.parallel() transform_executor = TransformExecutor( self.transform_evaluator_registry, self.evaluation_context, - committed_bundle, consumer_applied_transform, on_complete, + committed_bundle, consumer_applied_ptransform, on_complete, transform_executor_service) + print 'NEW TRANSFORMEXECUTOR', transform_executor, consumer_applied_ptransform, committed_bundle transform_executor_service.schedule(transform_executor) class _TypedUpdateQueue(object): @@ -421,11 +444,13 @@ def offer(self, item): class _ExecutorUpdate(object): """An internal status update on the state of the executor.""" - def __init__(self, unprocessed_elements=None, committed_bundle=None, exception=None): + def __init__(self, transform_executor=None, committed_bundle=None, unprocessed_bundle=None, exception=None): + assert transform_executor # Exactly one of them should be not-None - assert bool(produced_bundle) != bool(exception) - self.unprocessed_elements = unprocessed_elements or [] + assert sum([bool(committed_bundle), bool(unprocessed_bundle), bool(exception)]) == 1 + self.transform_executor = transform_executor self.committed_bundle = committed_bundle + self.unprocessed_bundle = unprocessed_bundle self.exception = exception self.exc_info = sys.exc_info() if self.exc_info[1] is not exception: @@ -456,10 +481,19 @@ def name(self): def call(self): try: + print 'YO' update = self._executor.all_updates.poll() + print 'YO', update while update: + print 'UPDATEE', update if update.committed_bundle: + print 'UPDATE', update self._executor.schedule_consumers(update.committed_bundle) + elif update.unprocessed_bundle: + print 'update.unprocessed_bundle', update.unprocessed_bundle.pcollection + self._executor.schedule_unprocessed_bundle( + update.transform_executor._applied_ptransform, + update.unprocessed_bundle) else: assert update.exception logging.warning('A task failed with exception.\n %s', @@ -472,11 +506,14 @@ def call(self): self._executor.executor_service) self._add_work_if_necessary(self._fire_timers()) except Exception as e: # pylint: disable=broad-except + import traceback + print traceback.format_exc() logging.error('Monitor task died due to exception.\n %s', e) self._executor.visible_updates.offer( _ExecutorServiceParallelExecutor._VisibleExecutorUpdate( sys.exc_info())) finally: + print 'SHUTDOWN?', self._should_shutdown() if not self._should_shutdown(): self._executor.executor_service.submit(self) @@ -503,9 +540,11 @@ def _should_shutdown(self): return False else: if self._executor.evaluation_context.is_done(): + print 'EXECTURO DONE' self._executor.visible_updates.offer( _ExecutorServiceParallelExecutor._VisibleExecutorUpdate()) else: + print 'EXECTURO: Nothing is scheduled for execution, but watermarks incomplete.' # Nothing is scheduled for execution, but watermarks incomplete. self._executor.visible_updates.offer( _ExecutorServiceParallelExecutor._VisibleExecutorUpdate( @@ -570,8 +609,15 @@ def _add_work_if_necessary(self, timers_fired): # All current TransformExecutors are blocked; add more work from the # roots. - for applied_transform in self._executor.root_nodes: - if not self._executor.evaluation_context.is_done(applied_transform): - self._executor.schedule_consumption( - applied_transform, None, - self._executor.default_completion_callback) + for applied_ptransform in self._executor.root_nodes: + if not self._executor.evaluation_context.is_done(applied_ptransform): + print 'SCHEDULE_CONSUMPTION', applied_ptransform + if not self._executor.root_nodes_to_pending_bundles[applied_ptransform]: + logging.warning('Root node %s not completed, but has no pending bundles.', applied_ptransform) + for bundle in self._executor.root_nodes_to_pending_bundles[applied_ptransform]: + print 'SCHEDULE', applied_ptransform, bundle + self._executor.schedule_consumption( + applied_ptransform, bundle, + self._executor.default_completion_callback) + # TODO: synchronization stuff + self._executor.root_nodes_to_pending_bundles[applied_ptransform] = [] diff --git a/sdks/python/apache_beam/runners/direct/transform_evaluator.py b/sdks/python/apache_beam/runners/direct/transform_evaluator.py index 7f89d0bbb29a..cc7d604b7674 100644 --- a/sdks/python/apache_beam/runners/direct/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/direct/transform_evaluator.py @@ -25,6 +25,7 @@ from apache_beam import pvalue from apache_beam.internal import pickler import apache_beam.io as io +from apache_beam.options.pipeline_options import TypeOptions from apache_beam.runners.common import DoFnRunner from apache_beam.runners.common import DoFnState from apache_beam.runners.direct.watermark_manager import WatermarkManager @@ -32,13 +33,48 @@ from apache_beam.runners.dataflow.native_io.iobase import _NativeWrite # pylint: disable=protected-access from apache_beam.testing.test_stream import TestStream from apache_beam.transforms import core +from apache_beam.transforms.timeutil import MIN_TIMESTAMP from apache_beam.transforms.window import GlobalWindows from apache_beam.transforms.window import WindowedValue from apache_beam.typehints.typecheck import OutputCheckWrapperDoFn from apache_beam.typehints.typecheck import TypeCheckError from apache_beam.typehints.typecheck import TypeCheckWrapperDoFn from apache_beam.utils import counters -from apache_beam.options.pipeline_options import TypeOptions +from apache_beam.utils.test_stream import ElementEvent +from apache_beam.utils.test_stream import WatermarkEvent +from apache_beam.utils.test_stream import ProcessingTimeEvent + +class RootBundleProvider(object): + """Provides bundles for the initial execution of a root transform.""" + def __init__(self, evaluation_context, applied_ptransform): + self._evaluation_context = evaluation_context + self._applied_ptransform = applied_ptransform + + def get_root_bundles(self): + raise NotImplementedError + +class DefaultRootBundleProvider(RootBundleProvider): + """Provides an empty bundle by default for root transforms.""" + + def get_root_bundles(self, ): + empty_bundle = ( + self._evaluation_context.create_empty_committed_bundle( + self._applied_ptransform.inputs[0])) + return [empty_bundle] + +class _TestStreamRootBundleProvider(RootBundleProvider): + + def get_root_bundles(self): + test_stream = self._applied_ptransform.transform + bundles = [] + if len(test_stream.events) > 0: + bundle = self._evaluation_context.create_bundle( + self._applied_ptransform.inputs[0]) + # Explicitly set timestamp to MIN_TIMESTAMP to ensure that we hold the watermark. + bundle.add(GlobalWindows.windowed_value(0, timestamp=MIN_TIMESTAMP)) + bundle.commit(None) + bundles.append(bundle) + return bundles class TransformEvaluatorRegistry(object): @@ -58,6 +94,10 @@ def __init__(self, evaluation_context): _NativeWrite: _NativeWriteEvaluator, TestStream: _TestStreamEvaluator, } + self._root_bundle_providers = { + core.PTransform: DefaultRootBundleProvider, + TestStream: _TestStreamRootBundleProvider, + } def for_application( self, applied_ptransform, input_committed_bundle, @@ -81,6 +121,18 @@ def for_application( input_committed_bundle, side_inputs, scoped_metrics_container) + def get_root_bundle_provider(self, applied_ptransform): + provider_cls = None + for cls in applied_ptransform.transform.__class__.mro(): + provider_cls = self._root_bundle_providers.get(cls) + if provider_cls: + break + if not provider_cls: + raise NotImplementedError( + 'Root provider for [%s] not implemented in runner %s' % ( + type(applied_ptransform.transform), self)) + return provider_cls(self._evaluation_context, applied_ptransform) + def should_execute_serially(self, applied_ptransform): """Returns True if this applied_ptransform should run one bundle at a time. @@ -217,17 +269,65 @@ class _TestStreamEvaluator(_TransformEvaluator): def __init__(self, evaluation_context, applied_ptransform, input_committed_bundle, side_inputs, scoped_metrics_container): - assert not input_committed_bundle + # TODO: figure out why timers fire + # assert not input_committed_bundle, input_committed_bundle assert not side_inputs + self.test_stream = applied_ptransform.transform super(_TestStreamEvaluator, self).__init__( evaluation_context, applied_ptransform, input_committed_bundle, side_inputs, scoped_metrics_container) + + def start_bundle(self): + # TODO: currently, for the first pass through the TestStream, no elements + # are passed through--we can therefore assume that the current index is 0 + # in this case; otherwise, the index will be stored on the process_element + # call. + self.current_index = -1 + self.watermark = MIN_TIMESTAMP + + def process_element(self, element): + index = element.value + self.watermark = element.timestamp + assert isinstance(index, int) + assert 0 <= index <= len(self.test_stream.events) + self.current_index = index def finish_bundle(self): assert len(self._outputs) == 1 output_pcollection = list(self._outputs)[0] - pass + # TODO: move to process_element once we provide a root bundle. + bundles = [] + event = self.test_stream.events[self.current_index] + watermark = self.watermark + if isinstance(event, ElementEvent): + bundle = self._evaluation_context.create_bundle(output_pcollection) + for tv in event.timestamped_values: + bundle.output( + GlobalWindows.windowed_value(tv.value, timestamp=tv.timestamp)) + bundles.append(bundle) + elif isinstance(event, WatermarkEvent): + assert event.new_watermark >= self.watermark + watermark = event.new_watermark + elif isinstance(event, ProcessingTimeEvent): + # TODO: advance processing time in the context's mock clock. + pass + else: + raise ValueError('Invalid TestStream event: %s.' % event) + + unprocessed_bundle = None + if self.current_index < len(self.test_stream.events) - 1: + unprocessed_bundle = self._evaluation_context.create_bundle( + self._applied_ptransform.inputs[0]) + unprocessed_bundle.add(GlobalWindows.windowed_value(self.current_index + 1, timestamp=watermark)) + + + + print 'FINISH_BUNDLE', self.current_index, event, unprocessed_bundle + + return TransformResult( + self._applied_ptransform, bundles, unprocessed_bundle, None, None, + None, None) class _FlattenEvaluator(_TransformEvaluator): diff --git a/sdks/python/apache_beam/runners/direct/transform_result.py b/sdks/python/apache_beam/runners/direct/transform_result.py index 97ac4aaf6255..4e54303df157 100644 --- a/sdks/python/apache_beam/runners/direct/transform_result.py +++ b/sdks/python/apache_beam/runners/direct/transform_result.py @@ -26,12 +26,12 @@ class TransformResult(object): The result of evaluating an AppliedPTransform with a TransformEvaluator.""" def __init__(self, applied_ptransform, uncommitted_output_bundles, - unprocessed_elements, state, + unprocessed_bundle, state, timer_update, counters, watermark_hold, undeclared_tag_values=None): self.transform = applied_ptransform self.uncommitted_output_bundles = uncommitted_output_bundles - self.unprocessed_elements = unprocessed_elements or [] + self.unprocessed_bundle = unprocessed_bundle self.state = state # TODO: timer update is currently unused. self.timer_update = timer_update diff --git a/sdks/python/apache_beam/runners/direct/watermark_manager.py b/sdks/python/apache_beam/runners/direct/watermark_manager.py index 3a135397e12f..d2048a940e7d 100644 --- a/sdks/python/apache_beam/runners/direct/watermark_manager.py +++ b/sdks/python/apache_beam/runners/direct/watermark_manager.py @@ -88,16 +88,19 @@ def get_watermarks(self, applied_ptransform): return self._transform_to_watermarks[applied_ptransform] - def update_watermarks(self, completed_committed_bundle, applied_ptransform, + def update_watermarks(self, completed_committed_bundle, unprocessed_bundle, + applied_ptransform, timer_update, outputs, earliest_hold): assert isinstance(applied_ptransform, pipeline.AppliedPTransform) self._update_pending( - completed_committed_bundle, applied_ptransform, timer_update, outputs) + completed_committed_bundle, unprocessed_bundle, applied_ptransform, + timer_update, outputs) tw = self.get_watermarks(applied_ptransform) tw.hold(earliest_hold) self._refresh_watermarks(applied_ptransform) - def _update_pending(self, input_committed_bundle, applied_ptransform, + def _update_pending(self, input_committed_bundle, unprocessed_bundle, + applied_ptransform, timer_update, output_committed_bundles): """Updated list of pending bundles for the given AppliedPTransform.""" @@ -114,6 +117,11 @@ def _update_pending(self, input_committed_bundle, applied_ptransform, completed_tw = self._transform_to_watermarks[applied_ptransform] completed_tw.update_timers(timer_update) + if unprocessed_bundle: + print 'DOING completed_tw.add_pending(unprocessed_bundle)', completed_tw, unprocessed_bundle + print completed_tw._input_watermark + completed_tw.add_pending(unprocessed_bundle) + assert input_committed_bundle or applied_ptransform in self._root_transforms if input_committed_bundle and input_committed_bundle.has_elements(): completed_tw.remove_pending(input_committed_bundle) @@ -193,9 +201,17 @@ def remove_pending(self, completed): def refresh(self): with self._lock: + # TODO(ccy): fix for streaming pending_holder = (WatermarkManager.WATERMARK_NEG_INF if self._pending else WatermarkManager.WATERMARK_POS_INF) + # pending_holder = WatermarkManager.WATERMARK_NEG_INF + # for input_bundle in self._pending: + # # TODO: Perhaps we can have the Bundle class keep track of the minimum + # # timestamp so we don't have to do an iteration here. + # bundle_min_timestamp = min(wv.timestamp for wv in input_bundle.get_elements_iterable()) + # if bundle_min_timestamp < pending_holder: + # pending_holder = bundle_min_timestamp input_watermarks = [ tw.output_watermark for tw in self._input_transform_watermarks] @@ -207,6 +223,8 @@ def refresh(self): new_output_watermark = min(self._input_watermark, self._earliest_hold) advanced = new_output_watermark > self._output_watermark + if advanced: + print 'ADVANCED', new_output_watermark, 'from', self._output_watermark self._output_watermark = new_output_watermark return advanced From fc661f6e22fb2d4b648af345527ed67a32776754 Mon Sep 17 00:00:00 2001 From: Charles Chen Date: Wed, 24 May 2017 14:54:09 -0700 Subject: [PATCH 03/21] Slightly better watermark propagation, debugging print cleanup. --- .../runners/direct/bundle_factory.py | 3 +- .../runners/direct/evaluation_context.py | 1 - .../apache_beam/runners/direct/executor.py | 14 ++------- .../runners/direct/streaming_test.py | 13 ++++++-- .../runners/direct/transform_evaluator.py | 4 ++- .../runners/direct/watermark_manager.py | 31 +++++++++---------- 6 files changed, 32 insertions(+), 34 deletions(-) diff --git a/sdks/python/apache_beam/runners/direct/bundle_factory.py b/sdks/python/apache_beam/runners/direct/bundle_factory.py index a46d10904d95..3c979ff051d1 100644 --- a/sdks/python/apache_beam/runners/direct/bundle_factory.py +++ b/sdks/python/apache_beam/runners/direct/bundle_factory.py @@ -108,8 +108,7 @@ def windowed_values(self): self._initial_windowed_value.windows) def __init__(self, pcollection, stacked=True): - # assert isinstance(pcollection, pvalue.PCollection) - print 'INIT PCOLLECTION', pcollection + assert isinstance(pcollection, (pvalue.PBegin, pvalue.PCollection)) self._pcollection = pcollection self._elements = [] self._stacked = stacked diff --git a/sdks/python/apache_beam/runners/direct/evaluation_context.py b/sdks/python/apache_beam/runners/direct/evaluation_context.py index f992bd09600f..062d01e9df7c 100644 --- a/sdks/python/apache_beam/runners/direct/evaluation_context.py +++ b/sdks/python/apache_beam/runners/direct/evaluation_context.py @@ -202,7 +202,6 @@ def handle_result( committed_bundles, unprocessed_bundle = self._commit_bundles( result.uncommitted_output_bundles, result.unprocessed_bundle) - print 'UNP', unprocessed_bundle self._watermark_manager.update_watermarks( completed_bundle, unprocessed_bundle, result.transform, completed_timers, committed_bundles, result.watermark_hold) diff --git a/sdks/python/apache_beam/runners/direct/executor.py b/sdks/python/apache_beam/runners/direct/executor.py index 4462c7fc4758..9e80709eb3f4 100644 --- a/sdks/python/apache_beam/runners/direct/executor.py +++ b/sdks/python/apache_beam/runners/direct/executor.py @@ -159,6 +159,7 @@ class _SerialEvaluationState(_TransformEvaluationState): """ def __init__(self, executor_service, scheduled): + raise Exception('wtf') super(_SerialEvaluationState, self).__init__(executor_service, scheduled) self.serial_queue = collections.deque() self.currently_evaluating = None @@ -227,7 +228,6 @@ def __init__(self, evaluation_context, all_updates, timers=None): self._timers = timers def handle_result(self, transform_executor, input_committed_bundle, transform_result): - print 'COMPLETIONCALLBACK.handle_result' output_committed_bundles = self._evaluation_context.handle_result( input_committed_bundle, self._timers, transform_result) for output_committed_bundle in output_committed_bundles: @@ -293,7 +293,7 @@ def call(self): for side_input in self._applied_ptransform.side_inputs] try: - print 'EVALLLL' + print '[!] Running evaluator for %s (input bundle: %s).' % (self._applied_ptransform, list(self._input_bundle.get_elements_iterable())) evaluator = self._transform_evaluator_registry.for_application( self._applied_ptransform, self._input_bundle, side_input_values, scoped_metrics_container) @@ -481,16 +481,12 @@ def name(self): def call(self): try: - print 'YO' update = self._executor.all_updates.poll() - print 'YO', update while update: - print 'UPDATEE', update + # print 'UPDATEE', update if update.committed_bundle: - print 'UPDATE', update self._executor.schedule_consumers(update.committed_bundle) elif update.unprocessed_bundle: - print 'update.unprocessed_bundle', update.unprocessed_bundle.pcollection self._executor.schedule_unprocessed_bundle( update.transform_executor._applied_ptransform, update.unprocessed_bundle) @@ -513,7 +509,6 @@ def call(self): _ExecutorServiceParallelExecutor._VisibleExecutorUpdate( sys.exc_info())) finally: - print 'SHUTDOWN?', self._should_shutdown() if not self._should_shutdown(): self._executor.executor_service.submit(self) @@ -540,7 +535,6 @@ def _should_shutdown(self): return False else: if self._executor.evaluation_context.is_done(): - print 'EXECTURO DONE' self._executor.visible_updates.offer( _ExecutorServiceParallelExecutor._VisibleExecutorUpdate()) else: @@ -611,11 +605,9 @@ def _add_work_if_necessary(self, timers_fired): # roots. for applied_ptransform in self._executor.root_nodes: if not self._executor.evaluation_context.is_done(applied_ptransform): - print 'SCHEDULE_CONSUMPTION', applied_ptransform if not self._executor.root_nodes_to_pending_bundles[applied_ptransform]: logging.warning('Root node %s not completed, but has no pending bundles.', applied_ptransform) for bundle in self._executor.root_nodes_to_pending_bundles[applied_ptransform]: - print 'SCHEDULE', applied_ptransform, bundle self._executor.schedule_consumption( applied_ptransform, bundle, self._executor.default_completion_callback) diff --git a/sdks/python/apache_beam/runners/direct/streaming_test.py b/sdks/python/apache_beam/runners/direct/streaming_test.py index 87cc8fdcadc1..f6133e65b786 100644 --- a/sdks/python/apache_beam/runners/direct/streaming_test.py +++ b/sdks/python/apache_beam/runners/direct/streaming_test.py @@ -19,7 +19,9 @@ import apache_beam as beam from apache_beam.utils.test_stream import * # TODO +from apache_beam.runners.direct.direct_runner import EagerRunner from apache_beam.runners.direct import DirectRunner +from apache_beam.transforms import window class StreamingTest(unittest.TestCase): @@ -30,10 +32,17 @@ def test_test_stream(self): .add_elements(['a', 'b', 'c']) .advance_watermark_to(20) .add_elements(['d']) - .advance_processing_time(10)) + .advance_processing_time(10) + .advance_watermark_to(300) + .add_elements([window.TimestampedValue('late', 12)])) p = beam.Pipeline(runner=DirectRunner()) elements = p | test_stream - groups = elements | beam.Map(lambda x: ('k', x)) | beam.GroupByKey() + groups = elements | beam.Map(lambda x: ('k', x))| beam.WindowInto(window.FixedWindows(15)) | beam.GroupByKey() + import sys + class PrintFn(beam.DoFn): + def process(self, element=beam.DoFn.ElementParam, timestamp=beam.DoFn.TimestampParam): + sys.stderr.write('!!!!!%s [ts=%s\n' % (element, timestamp)) + groups | beam.ParDo(PrintFn()) p.run(test_runner_api=False) diff --git a/sdks/python/apache_beam/runners/direct/transform_evaluator.py b/sdks/python/apache_beam/runners/direct/transform_evaluator.py index cc7d604b7674..a327805535c1 100644 --- a/sdks/python/apache_beam/runners/direct/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/direct/transform_evaluator.py @@ -152,6 +152,7 @@ def should_execute_serially(self, applied_ptransform): Returns: True if executor should execute applied_ptransform serially. """ + return False return isinstance(applied_ptransform.transform, (core._GroupByKeyOnly, _NativeWrite)) @@ -323,7 +324,7 @@ def finish_bundle(self): - print 'FINISH_BUNDLE', self.current_index, event, unprocessed_bundle + print '[!] TestStream finish_bundle', self.current_index, event, unprocessed_bundle return TransformResult( self._applied_ptransform, bundles, unprocessed_bundle, None, None, @@ -482,6 +483,7 @@ def start_bundle(self): self.key_coder = coders.registry.get_coder(kv_type_hint[0].tuple_types[0]) def process_element(self, element): + print '[!] GBK process_element', element assert not self.state.completed if (isinstance(element, WindowedValue) and isinstance(element.value, collections.Iterable) diff --git a/sdks/python/apache_beam/runners/direct/watermark_manager.py b/sdks/python/apache_beam/runners/direct/watermark_manager.py index d2048a940e7d..134eea852140 100644 --- a/sdks/python/apache_beam/runners/direct/watermark_manager.py +++ b/sdks/python/apache_beam/runners/direct/watermark_manager.py @@ -44,12 +44,12 @@ def __init__(self, clock, root_transforms, value_to_consumers): for root_transform in root_transforms: self._transform_to_watermarks[root_transform] = _TransformWatermarks( - self._clock) + self._clock, label=str(root_transform)) for consumers in value_to_consumers.values(): for consumer in consumers: self._transform_to_watermarks[consumer] = _TransformWatermarks( - self._clock) + self._clock, label=str(consumer)) for consumers in value_to_consumers.values(): for consumer in consumers: @@ -118,8 +118,6 @@ def _update_pending(self, input_committed_bundle, unprocessed_bundle, completed_tw.update_timers(timer_update) if unprocessed_bundle: - print 'DOING completed_tw.add_pending(unprocessed_bundle)', completed_tw, unprocessed_bundle - print completed_tw._input_watermark completed_tw.add_pending(unprocessed_bundle) assert input_committed_bundle or applied_ptransform in self._root_transforms @@ -152,7 +150,7 @@ def extract_fired_timers(self): class _TransformWatermarks(object): """Tracks input and output watermarks for aan AppliedPTransform.""" - def __init__(self, clock): + def __init__(self, clock, label=None): self._clock = clock self._input_transform_watermarks = [] self._input_watermark = WatermarkManager.WATERMARK_NEG_INF @@ -162,6 +160,9 @@ def __init__(self, clock): self._fired_timers = False self._lock = threading.Lock() + # TODO(ccy): remove debug label + self._label = label + def update_input_transform_watermarks(self, input_transform_watermarks): with self._lock: self._input_transform_watermarks = input_transform_watermarks @@ -201,17 +202,13 @@ def remove_pending(self, completed): def refresh(self): with self._lock: - # TODO(ccy): fix for streaming - pending_holder = (WatermarkManager.WATERMARK_NEG_INF - if self._pending else - WatermarkManager.WATERMARK_POS_INF) - # pending_holder = WatermarkManager.WATERMARK_NEG_INF - # for input_bundle in self._pending: - # # TODO: Perhaps we can have the Bundle class keep track of the minimum - # # timestamp so we don't have to do an iteration here. - # bundle_min_timestamp = min(wv.timestamp for wv in input_bundle.get_elements_iterable()) - # if bundle_min_timestamp < pending_holder: - # pending_holder = bundle_min_timestamp + pending_holder = WatermarkManager.WATERMARK_POS_INF + for input_bundle in self._pending: + # TODO: Perhaps we can have the Bundle class keep track of the minimum + # timestamp so we don't have to do an iteration here. + bundle_min_timestamp = min(wv.timestamp for wv in input_bundle.get_elements_iterable()) + if bundle_min_timestamp < pending_holder: + pending_holder = bundle_min_timestamp input_watermarks = [ tw.output_watermark for tw in self._input_transform_watermarks] @@ -224,7 +221,7 @@ def refresh(self): advanced = new_output_watermark > self._output_watermark if advanced: - print 'ADVANCED', new_output_watermark, 'from', self._output_watermark + print '[!] Watermark for', self._label, 'advanced', new_output_watermark, 'from', self._output_watermark, '(pending_holder:', pending_holder self._output_watermark = new_output_watermark return advanced From 053c7ae370df66d224469129f91fe8028ff3bcf4 Mon Sep 17 00:00:00 2001 From: Charles Chen Date: Wed, 24 May 2017 14:55:42 -0700 Subject: [PATCH 04/21] Factor out GroupAlsoByWindows so that it can be overridden by the DirectRunner. --- .../runners/direct/direct_runner.py | 3 + .../runners/direct/evaluation_context.py | 6 +- sdks/python/apache_beam/runners/runner.py | 3 + sdks/python/apache_beam/transforms/core.py | 93 ++++++++++--------- 4 files changed, 59 insertions(+), 46 deletions(-) diff --git a/sdks/python/apache_beam/runners/direct/direct_runner.py b/sdks/python/apache_beam/runners/direct/direct_runner.py index ecf5114feefd..90f6780cf2c8 100644 --- a/sdks/python/apache_beam/runners/direct/direct_runner.py +++ b/sdks/python/apache_beam/runners/direct/direct_runner.py @@ -56,6 +56,9 @@ def apply_CombinePerKey(self, transform, pcoll): except NotImplementedError: return transform.expand(pcoll) + def apply_GroupByKeya(self, transform, pcoll): + raise Exception('GABW!!!') + def run(self, pipeline): """Execute the entire pipeline and returns an DirectPipelineResult.""" diff --git a/sdks/python/apache_beam/runners/direct/evaluation_context.py b/sdks/python/apache_beam/runners/direct/evaluation_context.py index 062d01e9df7c..f5b25c8e8b12 100644 --- a/sdks/python/apache_beam/runners/direct/evaluation_context.py +++ b/sdks/python/apache_beam/runners/direct/evaluation_context.py @@ -147,7 +147,7 @@ def __init__(self, pipeline_options, bundle_factory, root_transforms, self._pcollection_to_views[view.pvalue].append(view) # AppliedPTransform -> Evaluator specific state objects - self._application_state_interals = {} + self._application_state_internals = {} self._watermark_manager = WatermarkManager( Clock(), root_transforms, value_to_consumers) self._side_inputs_container = _SideInputsContainer(views) @@ -232,7 +232,7 @@ def handle_result( counter.name, counter.combine_fn) merged_counter.accumulator.merge([counter.accumulator]) - self._application_state_interals[result.transform] = result.state + self._application_state_internals[result.transform] = result.state return committed_bundles def get_aggregator_values(self, aggregator_or_name): @@ -259,7 +259,7 @@ def _commit_bundles(self, uncommitted_output_bundles, unprocessed_bundle): def get_execution_context(self, applied_ptransform): return _ExecutionContext( self._watermark_manager.get_watermarks(applied_ptransform), - self._application_state_interals.get(applied_ptransform)) + self._application_state_internals.get(applied_ptransform)) def create_bundle(self, output_pcollection): """Create an uncommitted bundle for the specified PCollection.""" diff --git a/sdks/python/apache_beam/runners/runner.py b/sdks/python/apache_beam/runners/runner.py index af00d8f27ee0..77e845a1f7ca 100644 --- a/sdks/python/apache_beam/runners/runner.py +++ b/sdks/python/apache_beam/runners/runner.py @@ -145,8 +145,11 @@ def apply(self, transform, input): signature. """ for cls in transform.__class__.mro(): + print 'N', cls.__name__ m = getattr(self, 'apply_%s' % cls.__name__, None) if m: + if cls.__name__ != 'PTransform': + raise Exception('This is m: %s' % ((m(transform, input), cls),)) return m(transform, input) raise NotImplementedError( 'Execution of [%s] not implemented in runner %s.' % (transform, self)) diff --git a/sdks/python/apache_beam/transforms/core.py b/sdks/python/apache_beam/transforms/core.py index 0e497f9448d1..6f6f342e267c 100644 --- a/sdks/python/apache_beam/transforms/core.py +++ b/sdks/python/apache_beam/transforms/core.py @@ -1078,40 +1078,6 @@ def infer_output_type(self, input_type): key_type, value_type = trivial_inference.key_value_types(input_type) return Iterable[KV[key_type, typehints.WindowedValue[value_type]]] - class GroupAlsoByWindow(DoFn): - # TODO(robertwb): Support combiner lifting. - - def __init__(self, windowing): - super(GroupByKey.GroupAlsoByWindow, self).__init__() - self.windowing = windowing - - def infer_output_type(self, input_type): - key_type, windowed_value_iter_type = trivial_inference.key_value_types( - input_type) - value_type = windowed_value_iter_type.inner_type.inner_type - return Iterable[KV[key_type, Iterable[value_type]]] - - def start_bundle(self): - # pylint: disable=wrong-import-order, wrong-import-position - from apache_beam.transforms.trigger import InMemoryUnmergedState - from apache_beam.transforms.trigger import create_trigger_driver - # pylint: enable=wrong-import-order, wrong-import-position - self.driver = create_trigger_driver(self.windowing, True) - self.state_type = InMemoryUnmergedState - - def process(self, element): - k, vs = element - state = self.state_type() - # TODO(robertwb): Conditionally process in smaller chunks. - for wvalue in self.driver.process_elements(state, vs, MIN_TIMESTAMP): - yield wvalue.with_value((k, wvalue.value)) - while state.timers: - fired = state.get_and_clear_timers() - for timer_window, (name, time_domain, fire_time) in fired: - for wvalue in self.driver.process_timer( - timer_window, name, time_domain, fire_time, state): - yield wvalue.with_value((k, wvalue.value)) - def expand(self, pcoll): # This code path is only used in the local direct runner. For Dataflow # runner execution, the GroupByKey transform is expanded on the service. @@ -1136,17 +1102,14 @@ def expand(self, pcoll): | 'GroupByKey' >> (_GroupByKeyOnly() .with_input_types(reify_output_type) .with_output_types(gbk_input_type)) - | ('GroupByWindow' >> ParDo( - self.GroupAlsoByWindow(pcoll.windowing)) + | 'GroupByWindow' >> (GroupAlsoByWindow(pcoll.windowing) .with_input_types(gbk_input_type) .with_output_types(gbk_output_type))) - else: - # The input_type is None, run the default - return (pcoll - | 'ReifyWindows' >> ParDo(self.ReifyWindows()) - | 'GroupByKey' >> _GroupByKeyOnly() - | 'GroupByWindow' >> ParDo( - self.GroupAlsoByWindow(pcoll.windowing))) + # If the input_type is None, run the default + return (pcoll + | 'ReifyWindows' >> ParDo(self.ReifyWindows()) + | 'GroupByKey' >> _GroupByKeyOnly() + | 'GroupByWindow' >> GroupAlsoByWindow(pcoll.windowing)) @typehints.with_input_types(typehints.KV[K, V]) @@ -1162,6 +1125,50 @@ def expand(self, pcoll): return pvalue.PCollection(pcoll.pipeline) +class GroupAlsoByWindow(ParDo): + """The GroupAlsoByWindow transform.""" + + def __init__(self, windowing): + super(GroupAlsoByWindow, self).__init__( + GroupAlsoByWindowDoFn(windowing)) + + +class GroupAlsoByWindowDoFn(DoFn): + # TODO(robertwb): Support combiner lifting. + + def __init__(self, windowing): + super(GroupAlsoByWindowDoFn, self).__init__() + self.windowing = windowing + + def infer_output_type(self, input_type): + key_type, windowed_value_iter_type = trivial_inference.key_value_types( + input_type) + value_type = windowed_value_iter_type.inner_type.inner_type + return Iterable[KV[key_type, Iterable[value_type]]] + + def start_bundle(self): + # pylint: disable=wrong-import-order, wrong-import-position + from apache_beam.transforms.trigger import InMemoryUnmergedState + from apache_beam.transforms.trigger import create_trigger_driver + # pylint: enable=wrong-import-order, wrong-import-position + self.driver = create_trigger_driver(self.windowing) + self.state_type = InMemoryUnmergedState + + def process(self, element): + k, vs = element + state = self.state_type() + # TODO(robertwb): Conditionally process in smaller chunks. + for wvalue in self.driver.process_elements(state, vs, MIN_TIMESTAMP): + yield wvalue.with_value((k, wvalue.value)) + while state.timers: + fired = state.get_and_clear_timers() + for timer_window, (name, time_domain, fire_time) in fired: + for wvalue in self.driver.process_timer( + timer_window, name, time_domain, fire_time, state): + yield wvalue.with_value((k, wvalue.value)) + + + class Partition(PTransformWithSideInputs): """Split a PCollection into several partitions. From c2b3f3222baf5dd7b1f107dc0bfcc7f8b5d49111 Mon Sep 17 00:00:00 2001 From: Charles Chen Date: Fri, 21 Apr 2017 13:51:27 +0800 Subject: [PATCH 05/21] Don't buffer elements in GroupByKeyOnly. --- .../runners/direct/streaming_test.py | 1 + .../runners/direct/transform_evaluator.py | 55 +++++-------------- 2 files changed, 14 insertions(+), 42 deletions(-) diff --git a/sdks/python/apache_beam/runners/direct/streaming_test.py b/sdks/python/apache_beam/runners/direct/streaming_test.py index f6133e65b786..1175f9cd63ef 100644 --- a/sdks/python/apache_beam/runners/direct/streaming_test.py +++ b/sdks/python/apache_beam/runners/direct/streaming_test.py @@ -32,6 +32,7 @@ def test_test_stream(self): .add_elements(['a', 'b', 'c']) .advance_watermark_to(20) .add_elements(['d']) + .add_elements(['e']) .advance_processing_time(10) .advance_watermark_to(300) .add_elements([window.TimestampedValue('late', 12)])) diff --git a/sdks/python/apache_beam/runners/direct/transform_evaluator.py b/sdks/python/apache_beam/runners/direct/transform_evaluator.py index a327805535c1..c067adc15bff 100644 --- a/sdks/python/apache_beam/runners/direct/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/direct/transform_evaluator.py @@ -450,13 +450,6 @@ class _GroupByKeyOnlyEvaluator(_TransformEvaluator): MAX_ELEMENT_PER_BUNDLE = None - class _GroupByKeyOnlyEvaluatorState(object): - - def __init__(self): - # output: {} key -> [values] - self.output = collections.defaultdict(list) - self.completed = False - def __init__(self, evaluation_context, applied_ptransform, input_committed_bundle, side_inputs, scoped_metrics_container): assert not side_inputs @@ -464,15 +457,8 @@ def __init__(self, evaluation_context, applied_ptransform, evaluation_context, applied_ptransform, input_committed_bundle, side_inputs, scoped_metrics_container) - @property - def _is_final_bundle(self): - return (self._execution_context.watermarks.input_watermark - == WatermarkManager.WATERMARK_POS_INF) - def start_bundle(self): - self.state = (self._execution_context.existing_state - if self._execution_context.existing_state - else _GroupByKeyOnlyEvaluator._GroupByKeyOnlyEvaluatorState()) + self.gbk_items = collections.defaultdict(list) assert len(self._outputs) == 1 self.output_pcollection = list(self._outputs)[0] @@ -484,47 +470,32 @@ def start_bundle(self): def process_element(self, element): print '[!] GBK process_element', element - assert not self.state.completed if (isinstance(element, WindowedValue) and isinstance(element.value, collections.Iterable) and len(element.value) == 2): k, v = element.value - self.state.output[self.key_coder.encode(k)].append(v) + self.gbk_items[self.key_coder.encode(k)].append(v) else: raise TypeCheckError('Input to _GroupByKeyOnly must be a PCollection of ' 'windowed key-value pairs. Instead received: %r.' % element) def finish_bundle(self): - if self._is_final_bundle: - if self.state.completed: - # Ignore empty bundles after emitting output. (This may happen because - # empty bundles do not affect input watermarks.) - bundles = [] - else: - gbk_result = ( - map(GlobalWindows.windowed_value, ( - (self.key_coder.decode(k), v) - for k, v in self.state.output.iteritems()))) - - def len_element_fn(element): - _, v = element.value - return len(v) + gbk_result = ( + map(GlobalWindows.windowed_value, ( + (self.key_coder.decode(k), v) + for k, v in self.gbk_items.iteritems()))) - bundles = self._split_list_into_bundles( - self.output_pcollection, gbk_result, - _GroupByKeyOnlyEvaluator.MAX_ELEMENT_PER_BUNDLE, len_element_fn) + def len_element_fn(element): + _, v = element.value + return len(v) - self.state.completed = True - state = self.state - hold = WatermarkManager.WATERMARK_POS_INF - else: - bundles = [] - state = self.state - hold = WatermarkManager.WATERMARK_NEG_INF + bundles = self._split_list_into_bundles( + self.output_pcollection, gbk_result, + _GroupByKeyOnlyEvaluator.MAX_ELEMENT_PER_BUNDLE, len_element_fn) return TransformResult( - self._applied_ptransform, bundles, [], state, None, None, hold) + self._applied_ptransform, bundles, [], None, None, None, None) class _NativeWriteEvaluator(_TransformEvaluator): From 8854c53c1bc75631bd3b09b3f7ed6412eb93f844 Mon Sep 17 00:00:00 2001 From: Charles Chen Date: Wed, 24 May 2017 14:56:53 -0700 Subject: [PATCH 06/21] Skeleton implementation of GroupAlsoByKeyEvaluator. --- .../runners/direct/direct_runner.py | 6 +- .../runners/direct/transform_evaluator.py | 64 +++++++++++++++++++ sdks/python/apache_beam/runners/runner.py | 2 - sdks/python/apache_beam/transforms/core.py | 15 +++-- sdks/python/apache_beam/transforms/trigger.py | 2 +- 5 files changed, 77 insertions(+), 12 deletions(-) diff --git a/sdks/python/apache_beam/runners/direct/direct_runner.py b/sdks/python/apache_beam/runners/direct/direct_runner.py index 90f6780cf2c8..2a6336a71e64 100644 --- a/sdks/python/apache_beam/runners/direct/direct_runner.py +++ b/sdks/python/apache_beam/runners/direct/direct_runner.py @@ -26,6 +26,7 @@ import collections import logging +from apache_beam import pvalue from apache_beam.metrics.execution import MetricsEnvironment from apache_beam.runners.direct.bundle_factory import BundleFactory from apache_beam.runners.runner import PipelineResult @@ -56,8 +57,9 @@ def apply_CombinePerKey(self, transform, pcoll): except NotImplementedError: return transform.expand(pcoll) - def apply_GroupByKeya(self, transform, pcoll): - raise Exception('GABW!!!') + def apply_GroupAlsoByWindow(self, transform, pcoll): + transform._check_pcollection(pcoll) + return pvalue.PCollection(pcoll.pipeline) def run(self, pipeline): """Execute the entire pipeline and returns an DirectPipelineResult.""" diff --git a/sdks/python/apache_beam/runners/direct/transform_evaluator.py b/sdks/python/apache_beam/runners/direct/transform_evaluator.py index c067adc15bff..f68c2034df6b 100644 --- a/sdks/python/apache_beam/runners/direct/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/direct/transform_evaluator.py @@ -34,6 +34,8 @@ from apache_beam.testing.test_stream import TestStream from apache_beam.transforms import core from apache_beam.transforms.timeutil import MIN_TIMESTAMP +from apache_beam.transforms.trigger import InMemoryUnmergedState +from apache_beam.transforms.trigger import create_trigger_driver from apache_beam.transforms.window import GlobalWindows from apache_beam.transforms.window import WindowedValue from apache_beam.typehints.typecheck import OutputCheckWrapperDoFn @@ -91,6 +93,7 @@ def __init__(self, evaluation_context): core.Flatten: _FlattenEvaluator, core.ParDo: _ParDoEvaluator, core._GroupByKeyOnly: _GroupByKeyOnlyEvaluator, + core._GroupAlsoByWindow: _GroupAlsoByWindowEvaluator, _NativeWrite: _NativeWriteEvaluator, TestStream: _TestStreamEvaluator, } @@ -498,6 +501,67 @@ def len_element_fn(element): self._applied_ptransform, bundles, [], None, None, None, None) +class StateInternals(object): + pass + +class DirectStateInternals(StateInternals): + pass + +class _GroupAlsoByWindowEvaluator(_TransformEvaluator): + """TransformEvaluator for GroupByKeyOnly transform.""" + + MAX_ELEMENT_PER_BUNDLE = None + + def __init__(self, evaluation_context, applied_ptransform, + input_committed_bundle, side_inputs, scoped_metrics_container): + assert not side_inputs + super(_GroupAlsoByWindowEvaluator, self).__init__( + evaluation_context, applied_ptransform, input_committed_bundle, + side_inputs, scoped_metrics_container) + self.state = InMemoryUnmergedState() + + # self.driver = create_trigger_driver(self.windowing) + # self.state_type = InMemoryUnmergedState + + # def process(self, element): + # k, vs = element + # state = self.state_type() + # # TODO(robertwb): Conditionally process in smaller chunks. + # for wvalue in self.driver.process_elements(state, vs, MIN_TIMESTAMP): + # yield wvalue.with_value((k, wvalue.value)) + # while state.timers: + # fired = state.get_and_clear_timers() + # for timer_window, (name, time_domain, fire_time) in fired: + # for wvalue in self.driver.process_timer( + # timer_window, name, time_domain, fire_time, state): + # yield wvalue.with_value((k, wvalue.value)) + def start_bundle(self): + assert len(self._outputs) == 1 + self.output_pcollection = list(self._outputs)[0] + + self.driver = create_trigger_driver(self._applied_ptransform.transform.windowing) + + self.gabw_items = [] + + def process_element(self, element): + print '[!] GABW process_element', element + k, vs = element.value + # TODO(robertwb): Conditionally process in smaller chunks. + for wvalue in self.driver.process_elements(self.state, vs, MIN_TIMESTAMP): + self.gabw_items.append(wvalue.with_value((k, wvalue.value))) + # while state.timers: + # fired = state.get_and_clear_timers() + # for timer_window, (name, time_domain, fire_time) in fired: + # for wvalue in self.driver.process_timer( + # timer_window, name, time_domain, fire_time, state): + # yield wvalue.with_value((k, wvalue.value)) + + def finish_bundle(self): + print '[!] GABW_OUTPUT', self.gabw_items + return TransformResult( + self._applied_ptransform, bundles, [], None, None, None, None) + + class _NativeWriteEvaluator(_TransformEvaluator): """TransformEvaluator for _NativeWrite transform.""" diff --git a/sdks/python/apache_beam/runners/runner.py b/sdks/python/apache_beam/runners/runner.py index 77e845a1f7ca..55aff82c85e8 100644 --- a/sdks/python/apache_beam/runners/runner.py +++ b/sdks/python/apache_beam/runners/runner.py @@ -148,8 +148,6 @@ def apply(self, transform, input): print 'N', cls.__name__ m = getattr(self, 'apply_%s' % cls.__name__, None) if m: - if cls.__name__ != 'PTransform': - raise Exception('This is m: %s' % ((m(transform, input), cls),)) return m(transform, input) raise NotImplementedError( 'Execution of [%s] not implemented in runner %s.' % (transform, self)) diff --git a/sdks/python/apache_beam/transforms/core.py b/sdks/python/apache_beam/transforms/core.py index 6f6f342e267c..c0e2b1b35a0e 100644 --- a/sdks/python/apache_beam/transforms/core.py +++ b/sdks/python/apache_beam/transforms/core.py @@ -1102,14 +1102,14 @@ def expand(self, pcoll): | 'GroupByKey' >> (_GroupByKeyOnly() .with_input_types(reify_output_type) .with_output_types(gbk_input_type)) - | 'GroupByWindow' >> (GroupAlsoByWindow(pcoll.windowing) + | 'GroupByWindow' >> (_GroupAlsoByWindow(pcoll.windowing) .with_input_types(gbk_input_type) .with_output_types(gbk_output_type))) # If the input_type is None, run the default return (pcoll | 'ReifyWindows' >> ParDo(self.ReifyWindows()) | 'GroupByKey' >> _GroupByKeyOnly() - | 'GroupByWindow' >> GroupAlsoByWindow(pcoll.windowing)) + | 'GroupByWindow' >> _GroupAlsoByWindow(pcoll.windowing)) @typehints.with_input_types(typehints.KV[K, V]) @@ -1125,19 +1125,20 @@ def expand(self, pcoll): return pvalue.PCollection(pcoll.pipeline) -class GroupAlsoByWindow(ParDo): +class _GroupAlsoByWindow(ParDo): """The GroupAlsoByWindow transform.""" def __init__(self, windowing): - super(GroupAlsoByWindow, self).__init__( - GroupAlsoByWindowDoFn(windowing)) + super(_GroupAlsoByWindow, self).__init__( + _GroupAlsoByWindowDoFn(windowing)) + self.windowing = windowing -class GroupAlsoByWindowDoFn(DoFn): +class _GroupAlsoByWindowDoFn(DoFn): # TODO(robertwb): Support combiner lifting. def __init__(self, windowing): - super(GroupAlsoByWindowDoFn, self).__init__() + super(_GroupAlsoByWindowDoFn, self).__init__() self.windowing = windowing def infer_output_type(self, input_type): diff --git a/sdks/python/apache_beam/transforms/trigger.py b/sdks/python/apache_beam/transforms/trigger.py index 42009958552a..3dba890bc71b 100644 --- a/sdks/python/apache_beam/transforms/trigger.py +++ b/sdks/python/apache_beam/transforms/trigger.py @@ -1043,7 +1043,7 @@ def _output(self, window, finished, state): class InMemoryUnmergedState(UnmergedState): """In-memory implementation of UnmergedState. - Used for batch and testing. + Used for the DirectRunner and testing. """ def __init__(self, defensive_copy=True): # TODO(robertwb): Skip defensive_copy in production if it's too expensive. From 8be01a96055fb92013f6fc5aed8edc2af6e75e42 Mon Sep 17 00:00:00 2001 From: Charles Chen Date: Wed, 24 May 2017 14:57:30 -0700 Subject: [PATCH 07/21] Mostly working streaming GroupAlsoByWindow. --- .../apache_beam/runners/direct/executor.py | 16 ++++- .../runners/direct/transform_evaluator.py | 58 ++++++++++++++----- sdks/python/apache_beam/transforms/trigger.py | 2 + 3 files changed, 61 insertions(+), 15 deletions(-) diff --git a/sdks/python/apache_beam/runners/direct/executor.py b/sdks/python/apache_beam/runners/direct/executor.py index 9e80709eb3f4..7e1399d326e6 100644 --- a/sdks/python/apache_beam/runners/direct/executor.py +++ b/sdks/python/apache_beam/runners/direct/executor.py @@ -388,6 +388,7 @@ def schedule_consumers(self, committed_bundle): if committed_bundle.pcollection in self.value_to_consumers: consumers = self.value_to_consumers[committed_bundle.pcollection] for applied_ptransform in consumers: + print '[!] schedule_consumers', applied_ptransform, committed_bundle self.schedule_consumption(applied_ptransform, committed_bundle, self.default_completion_callback) @@ -415,6 +416,8 @@ def schedule_consumption(self, consumer_applied_ptransform, committed_bundle, committed_bundle, consumer_applied_ptransform, on_complete, transform_executor_service) print 'NEW TRANSFORMEXECUTOR', transform_executor, consumer_applied_ptransform, committed_bundle + # import traceback + # print '[!!!]', ''.join(' ' + x for x in traceback.format_stack()) transform_executor_service.schedule(transform_executor) class _TypedUpdateQueue(object): @@ -457,6 +460,14 @@ def __init__(self, transform_executor=None, committed_bundle=None, unprocessed_b # Not the right exception. self.exc_info = (exception, None, None) + def __repr__(self): + return '<_ExecutorUpdate transform_executor: %s, committed_bundle: %s, unprocessed_bundle: %s, exception: %s>' % ( + self.transform_executor, + self.committed_bundle, + self.unprocessed_bundle, + self.exception, + ) + class _VisibleExecutorUpdate(object): """An update of interest to the user. @@ -483,7 +494,7 @@ def call(self): try: update = self._executor.all_updates.poll() while update: - # print 'UPDATEE', update + # print '[!] UPDATE', update if update.committed_bundle: self._executor.schedule_consumers(update.committed_bundle) elif update.unprocessed_bundle: @@ -556,6 +567,7 @@ def _fire_timers(self): """ fired_timers = self._executor.evaluation_context.extract_fired_timers() for applied_ptransform in fired_timers: + print '[!] fired timer', applied_ptransform # Use an empty committed bundle. just to trigger. empty_bundle = ( self._executor.evaluation_context.create_empty_committed_bundle( @@ -566,6 +578,7 @@ def _fire_timers(self): self._executor.schedule_consumption( applied_ptransform, empty_bundle, timer_completion_callback) + # print '[!] fired_timers', bool(fired_timers) return bool(fired_timers) def _is_executing(self): @@ -608,6 +621,7 @@ def _add_work_if_necessary(self, timers_fired): if not self._executor.root_nodes_to_pending_bundles[applied_ptransform]: logging.warning('Root node %s not completed, but has no pending bundles.', applied_ptransform) for bundle in self._executor.root_nodes_to_pending_bundles[applied_ptransform]: + print '[!!] adding uncompleted bundle', applied_ptransform, bundle self._executor.schedule_consumption( applied_ptransform, bundle, self._executor.default_completion_callback) diff --git a/sdks/python/apache_beam/runners/direct/transform_evaluator.py b/sdks/python/apache_beam/runners/direct/transform_evaluator.py index f68c2034df6b..ea7d1dbdc84e 100644 --- a/sdks/python/apache_beam/runners/direct/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/direct/transform_evaluator.py @@ -472,7 +472,7 @@ def start_bundle(self): self.key_coder = coders.registry.get_coder(kv_type_hint[0].tuple_types[0]) def process_element(self, element): - print '[!] GBK process_element', element + # print '[!] GBK process_element', element if (isinstance(element, WindowedValue) and isinstance(element.value, collections.Iterable) and len(element.value) == 2): @@ -484,18 +484,20 @@ def process_element(self, element): % element) def finish_bundle(self): - gbk_result = ( - map(GlobalWindows.windowed_value, ( - (self.key_coder.decode(k), v) - for k, v in self.gbk_items.iteritems()))) + bundles = [] + if self.gbk_items: + gbk_result = ( + map(GlobalWindows.windowed_value, ( + (self.key_coder.decode(k), v) + for k, v in self.gbk_items.iteritems()))) - def len_element_fn(element): - _, v = element.value - return len(v) + def len_element_fn(element): + _, v = element.value + return len(v) - bundles = self._split_list_into_bundles( - self.output_pcollection, gbk_result, - _GroupByKeyOnlyEvaluator.MAX_ELEMENT_PER_BUNDLE, len_element_fn) + bundles = self._split_list_into_bundles( + self.output_pcollection, gbk_result, + _GroupByKeyOnlyEvaluator.MAX_ELEMENT_PER_BUNDLE, len_element_fn) return TransformResult( self._applied_ptransform, bundles, [], None, None, None, None) @@ -518,7 +520,6 @@ def __init__(self, evaluation_context, applied_ptransform, super(_GroupAlsoByWindowEvaluator, self).__init__( evaluation_context, applied_ptransform, input_committed_bundle, side_inputs, scoped_metrics_container) - self.state = InMemoryUnmergedState() # self.driver = create_trigger_driver(self.windowing) # self.state_type = InMemoryUnmergedState @@ -538,16 +539,38 @@ def __init__(self, evaluation_context, applied_ptransform, def start_bundle(self): assert len(self._outputs) == 1 self.output_pcollection = list(self._outputs)[0] + + self.keyed_states = (self._execution_context.existing_state + if self._execution_context.existing_state else collections.defaultdict(InMemoryUnmergedState)) self.driver = create_trigger_driver(self._applied_ptransform.transform.windowing) self.gabw_items = [] + # The input type of a GroupByKey will be KV[Any, Any] or more specific. + kv_type_hint = ( + self._applied_ptransform.transform.get_type_hints().input_types[0]) + self.key_coder = coders.registry.get_coder(kv_type_hint[0].tuple_types[0]) + # print 'GABW KEY_CODER', self.key_coder + def process_element(self, element): print '[!] GABW process_element', element k, vs = element.value + encoded_k = self.key_coder.encode(k) + state = self.keyed_states[encoded_k] + + print '[!] GABW current input watermark:', self._execution_context.watermarks.input_watermark + fired = state.get_and_clear_timers( + self._execution_context.watermarks.input_watermark) + print '[!!!] fired', fired + for timer_window, (name, time_domain, fire_time) in fired: + for wvalue in self.driver.process_timer( + timer_window, name, time_domain, fire_time, state): + # print '[!!] wvalue.value', wvalue.value + self.gabw_items.append(wvalue.with_value((k, wvalue.value))) + # TODO(robertwb): Conditionally process in smaller chunks. - for wvalue in self.driver.process_elements(self.state, vs, MIN_TIMESTAMP): + for wvalue in self.driver.process_elements(state, vs, MIN_TIMESTAMP): self.gabw_items.append(wvalue.with_value((k, wvalue.value))) # while state.timers: # fired = state.get_and_clear_timers() @@ -558,8 +581,15 @@ def process_element(self, element): def finish_bundle(self): print '[!] GABW_OUTPUT', self.gabw_items + bundle = self._evaluation_context.create_bundle( + self._applied_ptransform.inputs[0]) + for item in self.gabw_items: # TODO + bundle.add(item) + bundles = [] + if self.gabw_items: + qbundles = [bundle] return TransformResult( - self._applied_ptransform, bundles, [], None, None, None, None) + self._applied_ptransform, bundles, [], self.keyed_states, None, None, None) class _NativeWriteEvaluator(_TransformEvaluator): diff --git a/sdks/python/apache_beam/transforms/trigger.py b/sdks/python/apache_beam/transforms/trigger.py index 3dba890bc71b..6e2f6ebb7d18 100644 --- a/sdks/python/apache_beam/transforms/trigger.py +++ b/sdks/python/apache_beam/transforms/trigger.py @@ -1062,6 +1062,7 @@ def get_global_state(self, tag, default=None): return self.global_state.get(tag.tag, default) def set_timer(self, window, name, time_domain, timestamp): + print '[!!] set_timer', window, name, time_domain, timestamp self.timers[window][(name, time_domain)] = timestamp def clear_timer(self, window, name, time_domain): @@ -1105,6 +1106,7 @@ def clear_state(self, window, tag): def get_and_clear_timers(self, watermark=MAX_TIMESTAMP): expired = [] for window, timers in list(self.timers.items()): + print '[IIII] window, timers:', window, timers for (name, time_domain), timestamp in list(timers.items()): if timestamp <= watermark: expired.append((window, (name, time_domain, timestamp))) From bf38fe7afd219144e068d5dbd7035449f83bb723 Mon Sep 17 00:00:00 2001 From: Charles Chen Date: Wed, 24 May 2017 14:58:08 -0700 Subject: [PATCH 08/21] Introduce keyed bundles, KeyedWorkItems. --- .../runners/direct/bundle_factory.py | 8 +++-- .../runners/direct/evaluation_context.py | 27 ++++++++------- .../runners/direct/transform_evaluator.py | 33 ++++++++++--------- 3 files changed, 36 insertions(+), 32 deletions(-) diff --git a/sdks/python/apache_beam/runners/direct/bundle_factory.py b/sdks/python/apache_beam/runners/direct/bundle_factory.py index 3c979ff051d1..bdd8b67e7391 100644 --- a/sdks/python/apache_beam/runners/direct/bundle_factory.py +++ b/sdks/python/apache_beam/runners/direct/bundle_factory.py @@ -38,7 +38,10 @@ def __init__(self, stacked): self._stacked = stacked def create_bundle(self, output_pcollection): - return _Bundle(output_pcollection, self._stacked) + return _Bundle(output_pcollection, None, stacked=self._stacked) + + def create_keyed_bundle(self, output_pcollection, key): + return _Bundle(output_pcollection, key, stacked=self._stacked) def create_empty_committed_bundle(self, output_pcollection): bundle = self.create_bundle(output_pcollection) @@ -107,9 +110,10 @@ def windowed_values(self): yield WindowedValue(v, self._initial_windowed_value.timestamp, self._initial_windowed_value.windows) - def __init__(self, pcollection, stacked=True): + def __init__(self, pcollection, key, stacked=True): assert isinstance(pcollection, (pvalue.PBegin, pvalue.PCollection)) self._pcollection = pcollection + self.key = key self._elements = [] self._stacked = stacked self._committed = False diff --git a/sdks/python/apache_beam/runners/direct/evaluation_context.py b/sdks/python/apache_beam/runners/direct/evaluation_context.py index f5b25c8e8b12..a2a01a8d3e9f 100644 --- a/sdks/python/apache_beam/runners/direct/evaluation_context.py +++ b/sdks/python/apache_beam/runners/direct/evaluation_context.py @@ -32,17 +32,11 @@ class _ExecutionContext(object): - def __init__(self, watermarks, existing_state): - self._watermarks = watermarks - self._existing_state = existing_state - - @property - def watermarks(self): - return self._watermarks - - @property - def existing_state(self): - return self._existing_state + def __init__(self, watermarks, existing_state, key): + self.watermarks = watermarks + self.existing_state = existing_state + self.key = key + # TODO(ccy): key, clock as first arguments for consistency with Java. class _SideInputView(object): @@ -220,7 +214,7 @@ def handle_result( self._side_inputs_container.add_values( view, committed_bundle.get_elements_iterable(make_copy=True)) - if (self.get_execution_context(result.transform) + if (self.get_execution_context(result.transform, None) # TODO(ccy): what is the key here? .watermarks.input_watermark == WatermarkManager.WATERMARK_POS_INF): self._pending_unblocked_tasks.extend( @@ -256,15 +250,20 @@ def _commit_bundles(self, uncommitted_output_bundles, unprocessed_bundle): unprocessed_bundle.commit(None) return tuple(uncommitted_output_bundles), unprocessed_bundle - def get_execution_context(self, applied_ptransform): + def get_execution_context(self, applied_ptransform, key): return _ExecutionContext( self._watermark_manager.get_watermarks(applied_ptransform), - self._application_state_internals.get(applied_ptransform)) + self._application_state_internals.get(applied_ptransform), + key) def create_bundle(self, output_pcollection): """Create an uncommitted bundle for the specified PCollection.""" return self._bundle_factory.create_bundle(output_pcollection) + def create_keyed_bundle(self, output_pcollection, key): + """Create an uncommitted bundle for the specified PCollection.""" + return self._bundle_factory.create_keyed_bundle(output_pcollection, key) + def create_empty_committed_bundle(self, output_pcollection): """Create empty bundle useful for triggering evaluation.""" return self._bundle_factory.create_empty_committed_bundle( diff --git a/sdks/python/apache_beam/runners/direct/transform_evaluator.py b/sdks/python/apache_beam/runners/direct/transform_evaluator.py index ea7d1dbdc84e..6fc0e5c391b6 100644 --- a/sdks/python/apache_beam/runners/direct/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/direct/transform_evaluator.py @@ -171,7 +171,7 @@ def __init__(self, evaluation_context, applied_ptransform, self._side_inputs = side_inputs self._expand_outputs() self._execution_context = evaluation_context.get_execution_context( - applied_ptransform) + applied_ptransform, input_committed_bundle.key) self.scoped_metrics_container = scoped_metrics_container with scoped_metrics_container: self.start_bundle() @@ -447,6 +447,12 @@ def finish_bundle(self): self._applied_ptransform, bundles, [], None, None, result_counters, None, self._tagged_receivers.undeclared_in_memory_tag_values) +class KeyedWorkItem(object): + def __init__(self, key, timers=None, elements=None): + self.key = key + self.timers = timers + self.elements = elements + class _GroupByKeyOnlyEvaluator(_TransformEvaluator): """TransformEvaluator for _GroupByKeyOnly transform.""" @@ -485,19 +491,13 @@ def process_element(self, element): def finish_bundle(self): bundles = [] - if self.gbk_items: - gbk_result = ( - map(GlobalWindows.windowed_value, ( - (self.key_coder.decode(k), v) - for k, v in self.gbk_items.iteritems()))) - - def len_element_fn(element): - _, v = element.value - return len(v) + for encoded_k, vs in self.gbk_items.iteritems(): + k = self.key_coder.decode(encoded_k) + bundle = self._evaluation_context.create_keyed_bundle(self.output_pcollection, k) + kwi = KeyedWorkItem(k, elements=vs) + bundle.add(GlobalWindows.windowed_value(kwi)) + bundles.append(bundle) - bundles = self._split_list_into_bundles( - self.output_pcollection, gbk_result, - _GroupByKeyOnlyEvaluator.MAX_ELEMENT_PER_BUNDLE, len_element_fn) return TransformResult( self._applied_ptransform, bundles, [], None, None, None, None) @@ -539,7 +539,7 @@ def __init__(self, evaluation_context, applied_ptransform, def start_bundle(self): assert len(self._outputs) == 1 self.output_pcollection = list(self._outputs)[0] - + self.keyed_states = (self._execution_context.existing_state if self._execution_context.existing_state else collections.defaultdict(InMemoryUnmergedState)) @@ -554,8 +554,9 @@ def start_bundle(self): # print 'GABW KEY_CODER', self.key_coder def process_element(self, element): - print '[!] GABW process_element', element - k, vs = element.value + kwi = element.value + print '[!] GABW process_element', element, kwi + k, vs = kwi.key, kwi.elements encoded_k = self.key_coder.encode(k) state = self.keyed_states[encoded_k] From ae0d80d0b945e8c50426696495e893565dc7db4a Mon Sep 17 00:00:00 2001 From: Charles Chen Date: Mon, 24 Apr 2017 23:16:05 +0800 Subject: [PATCH 09/21] Rename old non-keyed state to legacy_state, introduce KeyedWorkItem, prepare to expose DirectUnmergedState API. --- .../runners/direct/evaluation_context.py | 55 ++++++++++++++++++- .../runners/direct/transform_evaluator.py | 34 +++++------- .../runners/direct/transform_result.py | 3 +- sdks/python/apache_beam/transforms/trigger.py | 2 + 4 files changed, 70 insertions(+), 24 deletions(-) diff --git a/sdks/python/apache_beam/runners/direct/evaluation_context.py b/sdks/python/apache_beam/runners/direct/evaluation_context.py index a2a01a8d3e9f..8934faad28bf 100644 --- a/sdks/python/apache_beam/runners/direct/evaluation_context.py +++ b/sdks/python/apache_beam/runners/direct/evaluation_context.py @@ -27,16 +27,60 @@ from apache_beam.runners.direct.watermark_manager import WatermarkManager from apache_beam.runners.direct.executor import TransformExecutor from apache_beam.runners.direct.direct_metrics import DirectMetrics +from apache_beam.transforms.trigger import InMemoryUnmergedState from apache_beam.utils import counters + + +class DirectUnmergedState(InMemoryUnmergedState): + def __init__(self): + super(DirectUnmergedState, self).__init__() + self.new_timers = [] + + def set_timer(self, window, name, time_domain, timestamp): + print '[!!] set_timer', window, name, time_domain, timestamp + super(DirectUnmergedState, self).set_timer(window, name, time_domain, timestamp) + + def clear_timer(self, window, name, time_domain): + print '[!!] clear_timer', window, name, time_domain + # TODO(ccy): this is not implemented, but is not strictly necessary as it is an optimization + super(DirectUnmergedState, self).clear_timer(window, name, time_domain) + + + +class StepContext(object): + def __init__(self, execution_context): + self._execution_context = execution_context + self._state = None + + def get_state(self): + # TODO(ccy): consider using copy on write semantics so that work items can be retried. + if not self._state: + if self._execution_context.existing_state: + self._state = self._execution_context.existing_state + else: + self._state = DirectUnmergedState() + return self._state + + + class _ExecutionContext(object): - def __init__(self, watermarks, existing_state, key): + def __init__(self, applied_ptransform, watermarks, existing_state, legacy_existing_state, key): + self.applied_ptransform = applied_ptransform self.watermarks = watermarks self.existing_state = existing_state + self.legacy_existing_state = legacy_existing_state self.key = key # TODO(ccy): key, clock as first arguments for consistency with Java. + self._step_context = None + + def get_step_context(self): + if not self._step_context: + self._step_context = StepContext(self) + return self._step_context + class _SideInputView(object): @@ -141,6 +185,8 @@ def __init__(self, pipeline_options, bundle_factory, root_transforms, self._pcollection_to_views[view.pvalue].append(view) # AppliedPTransform -> Evaluator specific state objects + self._legacy_existing_state = {} + # (AppliedPTransform, key) -> DirectUnmergedState objects self._application_state_internals = {} self._watermark_manager = WatermarkManager( Clock(), root_transforms, value_to_consumers) @@ -226,7 +272,8 @@ def handle_result( counter.name, counter.combine_fn) merged_counter.accumulator.merge([counter.accumulator]) - self._application_state_internals[result.transform] = result.state + self._legacy_existing_state[result.transform] = result.legacy_state + self._application_state_internals[(result.transform, completed_bundle.key)] = result.state return committed_bundles def get_aggregator_values(self, aggregator_or_name): @@ -252,8 +299,10 @@ def _commit_bundles(self, uncommitted_output_bundles, unprocessed_bundle): def get_execution_context(self, applied_ptransform, key): return _ExecutionContext( + applied_ptransform, self._watermark_manager.get_watermarks(applied_ptransform), - self._application_state_internals.get(applied_ptransform), + self._application_state_internals.get((applied_ptransform, key)), + self._legacy_existing_state.get(applied_ptransform), key) def create_bundle(self, output_pcollection): diff --git a/sdks/python/apache_beam/runners/direct/transform_evaluator.py b/sdks/python/apache_beam/runners/direct/transform_evaluator.py index 6fc0e5c391b6..facc2441918d 100644 --- a/sdks/python/apache_beam/runners/direct/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/direct/transform_evaluator.py @@ -34,7 +34,6 @@ from apache_beam.testing.test_stream import TestStream from apache_beam.transforms import core from apache_beam.transforms.timeutil import MIN_TIMESTAMP -from apache_beam.transforms.trigger import InMemoryUnmergedState from apache_beam.transforms.trigger import create_trigger_driver from apache_beam.transforms.window import GlobalWindows from apache_beam.transforms.window import WindowedValue @@ -265,7 +264,7 @@ def _read_values_to_bundles(reader): bundles = _read_values_to_bundles(reader) return TransformResult( - self._applied_ptransform, bundles, [], None, None, None, None) + self._applied_ptransform, bundles, [], None, None, None, None, None) class _TestStreamEvaluator(_TransformEvaluator): @@ -330,7 +329,7 @@ def finish_bundle(self): print '[!] TestStream finish_bundle', self.current_index, event, unprocessed_bundle return TransformResult( - self._applied_ptransform, bundles, unprocessed_bundle, None, None, + self._applied_ptransform, bundles, unprocessed_bundle, None, None, None, None, None) @@ -355,7 +354,7 @@ def process_element(self, element): def finish_bundle(self): bundles = [self.bundle] return TransformResult( - self._applied_ptransform, bundles, [], None, None, None, None) + self._applied_ptransform, bundles, [], None, None, None, None, None) class _TaggedReceivers(dict): @@ -444,7 +443,7 @@ def finish_bundle(self): bundles = self._tagged_receivers.values() result_counters = self._counter_factory.get_counters() return TransformResult( - self._applied_ptransform, bundles, [], None, None, result_counters, + self._applied_ptransform, bundles, [], None, None, None, result_counters, None, self._tagged_receivers.undeclared_in_memory_tag_values) class KeyedWorkItem(object): @@ -453,6 +452,9 @@ def __init__(self, key, timers=None, elements=None): self.timers = timers self.elements = elements + def __repr__(self): + return '' % (self.key, self.timers, self.elements) + class _GroupByKeyOnlyEvaluator(_TransformEvaluator): """TransformEvaluator for _GroupByKeyOnly transform.""" @@ -500,14 +502,7 @@ def finish_bundle(self): return TransformResult( - self._applied_ptransform, bundles, [], None, None, None, None) - - -class StateInternals(object): - pass - -class DirectStateInternals(StateInternals): - pass + self._applied_ptransform, bundles, [], None, None, None, None, None) class _GroupAlsoByWindowEvaluator(_TransformEvaluator): """TransformEvaluator for GroupByKeyOnly transform.""" @@ -540,8 +535,7 @@ def start_bundle(self): assert len(self._outputs) == 1 self.output_pcollection = list(self._outputs)[0] - self.keyed_states = (self._execution_context.existing_state - if self._execution_context.existing_state else collections.defaultdict(InMemoryUnmergedState)) + self.state = self._execution_context.get_step_context().get_state() self.driver = create_trigger_driver(self._applied_ptransform.transform.windowing) @@ -558,7 +552,7 @@ def process_element(self, element): print '[!] GABW process_element', element, kwi k, vs = kwi.key, kwi.elements encoded_k = self.key_coder.encode(k) - state = self.keyed_states[encoded_k] + state = self.state print '[!] GABW current input watermark:', self._execution_context.watermarks.input_watermark fired = state.get_and_clear_timers( @@ -590,7 +584,7 @@ def finish_bundle(self): if self.gabw_items: qbundles = [bundle] return TransformResult( - self._applied_ptransform, bundles, [], self.keyed_states, None, None, None) + self._applied_ptransform, bundles, [], self.state, None, None, None, None) class _NativeWriteEvaluator(_TransformEvaluator): @@ -618,8 +612,8 @@ def _has_already_produced_output(self): def start_bundle(self): # state: [values] - self.state = (self._execution_context.existing_state - if self._execution_context.existing_state else []) + self.state = (self._execution_context.legacy_existing_state + if self._execution_context.legacy_existing_state else []) def process_element(self, element): self.state.append(element) @@ -647,4 +641,4 @@ def finish_bundle(self): hold = WatermarkManager.WATERMARK_NEG_INF return TransformResult( - self._applied_ptransform, [], [], state, None, None, hold) + self._applied_ptransform, [], [], None, state, None, None, hold) diff --git a/sdks/python/apache_beam/runners/direct/transform_result.py b/sdks/python/apache_beam/runners/direct/transform_result.py index 4e54303df157..153581478d42 100644 --- a/sdks/python/apache_beam/runners/direct/transform_result.py +++ b/sdks/python/apache_beam/runners/direct/transform_result.py @@ -26,13 +26,14 @@ class TransformResult(object): The result of evaluating an AppliedPTransform with a TransformEvaluator.""" def __init__(self, applied_ptransform, uncommitted_output_bundles, - unprocessed_bundle, state, + unprocessed_bundle, state, legacy_state, timer_update, counters, watermark_hold, undeclared_tag_values=None): self.transform = applied_ptransform self.uncommitted_output_bundles = uncommitted_output_bundles self.unprocessed_bundle = unprocessed_bundle self.state = state + self.legacy_state = legacy_state # TODO: timer update is currently unused. self.timer_update = timer_update self.counters = counters diff --git a/sdks/python/apache_beam/transforms/trigger.py b/sdks/python/apache_beam/transforms/trigger.py index 6e2f6ebb7d18..a4af966ed63c 100644 --- a/sdks/python/apache_beam/transforms/trigger.py +++ b/sdks/python/apache_beam/transforms/trigger.py @@ -1072,6 +1072,7 @@ def get_window(self, window_id): return window_id def add_state(self, window, tag, value): + print '[!!] add_state', window, tag, value if self.defensive_copy: value = copy.deepcopy(value) if isinstance(tag, _ValueStateTag): @@ -1119,3 +1120,4 @@ def __repr__(self): state_str = '\n'.join('%s: %s' % (key, dict(state)) for key, state in self.state.items()) return 'timers: %s\nstate: %s' % (dict(self.timers), state_str) + From 97de55600417145702576424dc7512de3f981838 Mon Sep 17 00:00:00 2001 From: Charles Chen Date: Wed, 24 May 2017 15:01:59 -0700 Subject: [PATCH 10/21] Fix rebase to head after Beam 2.0 release. --- sdks/python/apache_beam/runners/direct/executor.py | 3 ++- sdks/python/apache_beam/runners/direct/streaming_test.py | 2 +- .../apache_beam/runners/direct/transform_evaluator.py | 8 ++++---- 3 files changed, 7 insertions(+), 6 deletions(-) diff --git a/sdks/python/apache_beam/runners/direct/executor.py b/sdks/python/apache_beam/runners/direct/executor.py index 7e1399d326e6..4f246bc19cbf 100644 --- a/sdks/python/apache_beam/runners/direct/executor.py +++ b/sdks/python/apache_beam/runners/direct/executor.py @@ -237,7 +237,7 @@ def handle_result(self, transform_executor, input_committed_bundle, transform_re committed_bundle=output_committed_bundle)) if transform_result.unprocessed_bundle: self._all_updates.offer( - _ExecutorServiceParallelExecutor.ExecutorUpdate( + _ExecutorServiceParallelExecutor._ExecutorUpdate( transform_executor=transform_executor, unprocessed_bundle=transform_result.unprocessed_bundle)) return output_committed_bundles @@ -323,6 +323,7 @@ def call(self): self._completion_callback.handle_result(self, self._input_bundle, result) return result # TODO: not necessary? except Exception as e: # pylint: disable=broad-except + import traceback logging.warning('Task failed: %s', traceback.format_exc(), exc_info=True) self._completion_callback.handle_exception(self, e) finally: diff --git a/sdks/python/apache_beam/runners/direct/streaming_test.py b/sdks/python/apache_beam/runners/direct/streaming_test.py index 1175f9cd63ef..8b2f88c3fd26 100644 --- a/sdks/python/apache_beam/runners/direct/streaming_test.py +++ b/sdks/python/apache_beam/runners/direct/streaming_test.py @@ -18,9 +18,9 @@ import unittest import apache_beam as beam -from apache_beam.utils.test_stream import * # TODO from apache_beam.runners.direct.direct_runner import EagerRunner from apache_beam.runners.direct import DirectRunner +from apache_beam.testing.test_stream import * # TODO from apache_beam.transforms import window diff --git a/sdks/python/apache_beam/runners/direct/transform_evaluator.py b/sdks/python/apache_beam/runners/direct/transform_evaluator.py index facc2441918d..cd5ddb98cb13 100644 --- a/sdks/python/apache_beam/runners/direct/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/direct/transform_evaluator.py @@ -33,7 +33,7 @@ from apache_beam.runners.dataflow.native_io.iobase import _NativeWrite # pylint: disable=protected-access from apache_beam.testing.test_stream import TestStream from apache_beam.transforms import core -from apache_beam.transforms.timeutil import MIN_TIMESTAMP +from apache_beam.utils.timestamp import MIN_TIMESTAMP from apache_beam.transforms.trigger import create_trigger_driver from apache_beam.transforms.window import GlobalWindows from apache_beam.transforms.window import WindowedValue @@ -41,9 +41,9 @@ from apache_beam.typehints.typecheck import TypeCheckError from apache_beam.typehints.typecheck import TypeCheckWrapperDoFn from apache_beam.utils import counters -from apache_beam.utils.test_stream import ElementEvent -from apache_beam.utils.test_stream import WatermarkEvent -from apache_beam.utils.test_stream import ProcessingTimeEvent +from apache_beam.testing.test_stream import ElementEvent +from apache_beam.testing.test_stream import WatermarkEvent +from apache_beam.testing.test_stream import ProcessingTimeEvent class RootBundleProvider(object): """Provides bundles for the initial execution of a root transform.""" From 341fa9245e5d67cc5fab4244bfeca5ea49c72231 Mon Sep 17 00:00:00 2001 From: Charles Chen Date: Thu, 25 May 2017 14:22:03 -0700 Subject: [PATCH 11/21] Clarify transform_keyed_states. --- .../runners/direct/evaluation_context.py | 19 ++++++++++++++----- .../runners/direct/watermark_manager.py | 3 ++- 2 files changed, 16 insertions(+), 6 deletions(-) diff --git a/sdks/python/apache_beam/runners/direct/evaluation_context.py b/sdks/python/apache_beam/runners/direct/evaluation_context.py index 8934faad28bf..633a1c45adb3 100644 --- a/sdks/python/apache_beam/runners/direct/evaluation_context.py +++ b/sdks/python/apache_beam/runners/direct/evaluation_context.py @@ -186,10 +186,10 @@ def __init__(self, pipeline_options, bundle_factory, root_transforms, # AppliedPTransform -> Evaluator specific state objects self._legacy_existing_state = {} - # (AppliedPTransform, key) -> DirectUnmergedState objects - self._application_state_internals = {} + # AppliedPTransform -> {key -> DirectUnmergedState objects}; todo: rename + self._transform_keyed_states = self._initialize_transform_states(root_transforms, value_to_consumers) self._watermark_manager = WatermarkManager( - Clock(), root_transforms, value_to_consumers) + Clock(), root_transforms, value_to_consumers, self._transform_keyed_states) self._side_inputs_container = _SideInputsContainer(views) self._pending_unblocked_tasks = [] self._counter_factory = counters.CounterFactory() @@ -197,6 +197,15 @@ def __init__(self, pipeline_options, bundle_factory, root_transforms, self._metrics = DirectMetrics() self._lock = threading.Lock() + + def _initialize_transform_states(self, root_transforms, value_to_consumers): + transform_keyed_states = {} + for transform in root_transforms: + transform_keyed_states[transform] = {} + for consumers in value_to_consumers.values(): + for consumer in consumers: + transform_keyed_states[consumer] = {} + return transform_keyed_states def use_pvalue_cache(self, cache): assert not self._cache @@ -273,7 +282,7 @@ def handle_result( merged_counter.accumulator.merge([counter.accumulator]) self._legacy_existing_state[result.transform] = result.legacy_state - self._application_state_internals[(result.transform, completed_bundle.key)] = result.state + self._transform_keyed_states[result.transform][completed_bundle.key] = result.state return committed_bundles def get_aggregator_values(self, aggregator_or_name): @@ -301,7 +310,7 @@ def get_execution_context(self, applied_ptransform, key): return _ExecutionContext( applied_ptransform, self._watermark_manager.get_watermarks(applied_ptransform), - self._application_state_internals.get((applied_ptransform, key)), + self._transform_keyed_states[applied_ptransform].get(key), self._legacy_existing_state.get(applied_ptransform), key) diff --git a/sdks/python/apache_beam/runners/direct/watermark_manager.py b/sdks/python/apache_beam/runners/direct/watermark_manager.py index 134eea852140..7f1d1d6ae3ac 100644 --- a/sdks/python/apache_beam/runners/direct/watermark_manager.py +++ b/sdks/python/apache_beam/runners/direct/watermark_manager.py @@ -35,10 +35,11 @@ class WatermarkManager(object): WATERMARK_POS_INF = MAX_TIMESTAMP WATERMARK_NEG_INF = MIN_TIMESTAMP - def __init__(self, clock, root_transforms, value_to_consumers): + def __init__(self, clock, root_transforms, value_to_consumers, transform_keyed_states): self._clock = clock # processing time clock self._value_to_consumers = value_to_consumers self._root_transforms = root_transforms + self._transform_keyed_states = transform_keyed_states # AppliedPTransform -> TransformWatermarks self._transform_to_watermarks = {} From 1e83d5e900c7039ccf89d989081ae50b3b3e510f Mon Sep 17 00:00:00 2001 From: Charles Chen Date: Thu, 25 May 2017 17:38:31 -0700 Subject: [PATCH 12/21] Support timer firings. --- .../runners/direct/evaluation_context.py | 6 +- .../apache_beam/runners/direct/executor.py | 30 +++++++--- .../runners/direct/transform_evaluator.py | 7 ++- .../runners/direct/watermark_manager.py | 59 ++++++++++++++++--- sdks/python/apache_beam/transforms/core.py | 1 - sdks/python/apache_beam/transforms/trigger.py | 21 +++++-- 6 files changed, 99 insertions(+), 25 deletions(-) diff --git a/sdks/python/apache_beam/runners/direct/evaluation_context.py b/sdks/python/apache_beam/runners/direct/evaluation_context.py index 633a1c45adb3..eede6c4a59c3 100644 --- a/sdks/python/apache_beam/runners/direct/evaluation_context.py +++ b/sdks/python/apache_beam/runners/direct/evaluation_context.py @@ -282,7 +282,11 @@ def handle_result( merged_counter.accumulator.merge([counter.accumulator]) self._legacy_existing_state[result.transform] = result.legacy_state - self._transform_keyed_states[result.transform][completed_bundle.key] = result.state + if not result.state: + if completed_bundle.key in self._transform_keyed_states[result.transform]: + del self._transform_keyed_states[result.transform][completed_bundle.key] + else: + self._transform_keyed_states[result.transform][completed_bundle.key] = result.state return committed_bundles def get_aggregator_values(self, aggregator_or_name): diff --git a/sdks/python/apache_beam/runners/direct/executor.py b/sdks/python/apache_beam/runners/direct/executor.py index 4f246bc19cbf..9cd1a3311689 100644 --- a/sdks/python/apache_beam/runners/direct/executor.py +++ b/sdks/python/apache_beam/runners/direct/executor.py @@ -28,6 +28,7 @@ from apache_beam.metrics.execution import MetricsContainer from apache_beam.metrics.execution import ScopedMetricsContainer +from apache_beam.transforms.window import GlobalWindows class _ExecutorService(object): @@ -566,19 +567,34 @@ def _fire_timers(self): Returns: True if timers fired. """ + # TODO: refactor this out + from apache_beam.runners.direct.transform_evaluator import KeyedWorkItem fired_timers = self._executor.evaluation_context.extract_fired_timers() - for applied_ptransform in fired_timers: + for applied_ptransform, timer_firings in fired_timers: print '[!] fired timer', applied_ptransform - # Use an empty committed bundle. just to trigger. - empty_bundle = ( - self._executor.evaluation_context.create_empty_committed_bundle( - applied_ptransform.inputs[0])) + bundles = [] + for timer_firing in timer_firings: + if timer_firing is None: + # Use an empty committed bundle. just to trigger. + # TODO: are there still timer-using things that don't take keyed work items? + empty_bundle = ( + self._executor.evaluation_context.create_empty_committed_bundle( + applied_ptransform.inputs[0])) + bundles.append(empty_bundle) + else: + # TODO: we can potentially consolidate bundles if there are multiple timers firing for a single key. + bundle = self._executor.evaluation_context.create_keyed_bundle(applied_ptransform.inputs[0], timer_firing.key) + bundle.add(GlobalWindows.windowed_value(KeyedWorkItem(timer_firing.key, timers=[timer_firing]))) + bundle.commit(None) # TODO: synchronized processing time? + bundles.append(bundle) + timer_completion_callback = _CompletionCallback( self._executor.evaluation_context, self._executor.all_updates, applied_ptransform) - self._executor.schedule_consumption( - applied_ptransform, empty_bundle, timer_completion_callback) + for bundle in bundles: + self._executor.schedule_consumption( + applied_ptransform, bundle, timer_completion_callback) # print '[!] fired_timers', bool(fired_timers) return bool(fired_timers) diff --git a/sdks/python/apache_beam/runners/direct/transform_evaluator.py b/sdks/python/apache_beam/runners/direct/transform_evaluator.py index cd5ddb98cb13..31822cf676ec 100644 --- a/sdks/python/apache_beam/runners/direct/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/direct/transform_evaluator.py @@ -550,7 +550,7 @@ def start_bundle(self): def process_element(self, element): kwi = element.value print '[!] GABW process_element', element, kwi - k, vs = kwi.key, kwi.elements + k, timers, vs = kwi.key, kwi.timers, kwi.elements encoded_k = self.key_coder.encode(k) state = self.state @@ -565,8 +565,9 @@ def process_element(self, element): self.gabw_items.append(wvalue.with_value((k, wvalue.value))) # TODO(robertwb): Conditionally process in smaller chunks. - for wvalue in self.driver.process_elements(state, vs, MIN_TIMESTAMP): - self.gabw_items.append(wvalue.with_value((k, wvalue.value))) + if vs: + for wvalue in self.driver.process_elements(state, vs, MIN_TIMESTAMP): + self.gabw_items.append(wvalue.with_value((k, wvalue.value))) # while state.timers: # fired = state.get_and_clear_timers() # for timer_window, (name, time_domain, fire_time) in fired: diff --git a/sdks/python/apache_beam/runners/direct/watermark_manager.py b/sdks/python/apache_beam/runners/direct/watermark_manager.py index 7f1d1d6ae3ac..e1d0a975edc2 100644 --- a/sdks/python/apache_beam/runners/direct/watermark_manager.py +++ b/sdks/python/apache_beam/runners/direct/watermark_manager.py @@ -44,13 +44,15 @@ def __init__(self, clock, root_transforms, value_to_consumers, transform_keyed_s self._transform_to_watermarks = {} for root_transform in root_transforms: + keyed_states = self._transform_keyed_states[root_transform] self._transform_to_watermarks[root_transform] = _TransformWatermarks( - self._clock, label=str(root_transform)) + self._clock, keyed_states, label=str(root_transform)) for consumers in value_to_consumers.values(): for consumer in consumers: + keyed_states = self._transform_keyed_states[consumer] self._transform_to_watermarks[consumer] = _TransformWatermarks( - self._clock, label=str(consumer)) + self._clock, keyed_states, label=str(consumer)) for consumers in value_to_consumers.values(): for consumer in consumers: @@ -143,16 +145,18 @@ def _refresh_watermarks(self, applied_ptransform): def extract_fired_timers(self): all_timers = [] for applied_ptransform, tw in self._transform_to_watermarks.iteritems(): - if tw.extract_fired_timers(): - all_timers.append(applied_ptransform) + fired_timers = tw.extract_fired_timers() + if fired_timers: + all_timers.append((applied_ptransform, fired_timers)) return all_timers class _TransformWatermarks(object): """Tracks input and output watermarks for aan AppliedPTransform.""" - def __init__(self, clock, label=None): + def __init__(self, clock, keyed_states, label=None): self._clock = clock + self._keyed_states = keyed_states self._input_transform_watermarks = [] self._input_watermark = WatermarkManager.WATERMARK_NEG_INF self._output_watermark = WatermarkManager.WATERMARK_NEG_INF @@ -202,6 +206,8 @@ def remove_pending(self, completed): self._pending.remove(completed) def refresh(self): + # TODO: remove this and the below assert + from apache_beam.runners.direct.evaluation_context import DirectUnmergedState with self._lock: pending_holder = WatermarkManager.WATERMARK_POS_INF for input_bundle in self._pending: @@ -211,13 +217,21 @@ def refresh(self): if bundle_min_timestamp < pending_holder: pending_holder = bundle_min_timestamp + earliest_watermark_hold = WatermarkManager.WATERMARK_POS_INF + for unused_key, state in self._keyed_states.iteritems(): + assert isinstance(state, DirectUnmergedState), state + print '~~~~~~~~WHSTATE [key=', unused_key, ']', state + earliest_watermark_hold = state.get_earliest_hold() + print 'holds [current watermark =', self._input_watermark, ']:', state.get_earliest_hold() + + input_watermarks = [ tw.output_watermark for tw in self._input_transform_watermarks] input_watermarks.append(WatermarkManager.WATERMARK_POS_INF) producer_watermark = min(input_watermarks) self._input_watermark = max(self._input_watermark, - min(pending_holder, producer_watermark)) + min(pending_holder, producer_watermark, earliest_watermark_hold)) new_output_watermark = min(self._input_watermark, self._earliest_hold) advanced = new_output_watermark > self._output_watermark @@ -231,12 +245,39 @@ def synchronized_processing_output_time(self): return self._clock.time() def extract_fired_timers(self): + # TODO: remove this and the below assert + from apache_beam.runners.direct.evaluation_context import DirectUnmergedState with self._lock: if self._fired_timers: - return False + print 'FIRED_TIMERS? FALSE', self + # Wait until fired timers have been processed. + # return False + + fired_timers = [] + for key, state in self._keyed_states.iteritems(): + assert isinstance(state, DirectUnmergedState), state + print '~~~~~~~~STATE [key=', key, ']', state + timers = state.get_timers(watermark=self._input_watermark) + print 'timers:', timers + for expired in timers: + _, (name, time_domain, timestamp) = expired + fired_timers.append(TimerFiring(key, time_domain, timestamp)) + + + # TODO: check if this hack is still necessary. should_fire = ( self._earliest_hold < WatermarkManager.WATERMARK_POS_INF and self._input_watermark == WatermarkManager.WATERMARK_POS_INF) - self._fired_timers = should_fire - return should_fire + if should_fire: + fired_timers.append(None) # Sentinel representing legacy timer firing. + self._fired_timers = fired_timers + return fired_timers + +class TimerFiring(object): + def __init__(self, key, time_domain, timestamp): + # TODO: add time domain. + self.key = key + self.time_domain = time_domain + self.timestamp = timestamp + diff --git a/sdks/python/apache_beam/transforms/core.py b/sdks/python/apache_beam/transforms/core.py index c0e2b1b35a0e..b4919201deb7 100644 --- a/sdks/python/apache_beam/transforms/core.py +++ b/sdks/python/apache_beam/transforms/core.py @@ -1169,7 +1169,6 @@ def process(self, element): yield wvalue.with_value((k, wvalue.value)) - class Partition(PTransformWithSideInputs): """Split a PCollection into several partitions. diff --git a/sdks/python/apache_beam/transforms/trigger.py b/sdks/python/apache_beam/transforms/trigger.py index a4af966ed63c..42082c3bf387 100644 --- a/sdks/python/apache_beam/transforms/trigger.py +++ b/sdks/python/apache_beam/transforms/trigger.py @@ -1104,18 +1104,31 @@ def clear_state(self, window, tag): if not self.state[window]: self.state.pop(window, None) - def get_and_clear_timers(self, watermark=MAX_TIMESTAMP): + def get_timers(self, clear=False, watermark=MAX_TIMESTAMP): expired = [] for window, timers in list(self.timers.items()): - print '[IIII] window, timers:', window, timers + # print '[IIII] window, timers:', window, timers for (name, time_domain), timestamp in list(timers.items()): if timestamp <= watermark: expired.append((window, (name, time_domain, timestamp))) - del timers[(name, time_domain)] - if not timers: + if clear: + del timers[(name, time_domain)] + if not timers and clear: del self.timers[window] return expired + def get_and_clear_timers(self, watermark=MAX_TIMESTAMP): + return self.get_timers(clear=True, watermark=watermark) + + def get_earliest_hold(self): + earliest_hold = MAX_TIMESTAMP + for window, tagged_states in self.state.iteritems(): + # TODO: is this general enough? + if 'watermark' in tagged_states: + hold = min(tagged_states['watermark']) + earliest_hold = min(earliest_hold, hold) + return earliest_hold + def __repr__(self): state_str = '\n'.join('%s: %s' % (key, dict(state)) for key, state in self.state.items()) From 256309f2f3b01b230b258b8001a494d75e3e1713 Mon Sep 17 00:00:00 2001 From: Charles Chen Date: Thu, 25 May 2017 18:08:44 -0700 Subject: [PATCH 13/21] Reintroduce hacks from head. --- sdks/python/apache_beam/runners/direct/transform_evaluator.py | 2 +- sdks/python/apache_beam/runners/direct/watermark_manager.py | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/runners/direct/transform_evaluator.py b/sdks/python/apache_beam/runners/direct/transform_evaluator.py index 31822cf676ec..023227948ec9 100644 --- a/sdks/python/apache_beam/runners/direct/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/direct/transform_evaluator.py @@ -235,7 +235,7 @@ class _BoundedReadEvaluator(_TransformEvaluator): def __init__(self, evaluation_context, applied_ptransform, input_committed_bundle, side_inputs, scoped_metrics_container): - assert not input_committed_bundle + # assert not input_committed_bundle, input_committed_bundle assert not side_inputs self._source = applied_ptransform.transform.source self._source.pipeline_options = evaluation_context.pipeline_options diff --git a/sdks/python/apache_beam/runners/direct/watermark_manager.py b/sdks/python/apache_beam/runners/direct/watermark_manager.py index e1d0a975edc2..eb7f61859f15 100644 --- a/sdks/python/apache_beam/runners/direct/watermark_manager.py +++ b/sdks/python/apache_beam/runners/direct/watermark_manager.py @@ -175,7 +175,8 @@ def update_input_transform_watermarks(self, input_transform_watermarks): def update_timers(self, timer_update): with self._lock: if timer_update: - assert self._fired_timers + # TODO: fix hack--we currently are able to produce several timer bundles, but the first one will clear + # assert self._fired_timers self._fired_timers = False @property From 25ce9f18a124e8d0aa80e05a516699da6a067c19 Mon Sep 17 00:00:00 2001 From: Charles Chen Date: Fri, 26 May 2017 14:12:59 -0700 Subject: [PATCH 14/21] Add debugging output. --- .../apache_beam/runners/direct/evaluation_context.py | 1 + sdks/python/apache_beam/runners/direct/executor.py | 1 + .../apache_beam/runners/direct/watermark_manager.py | 11 +++++++++++ 3 files changed, 13 insertions(+) diff --git a/sdks/python/apache_beam/runners/direct/evaluation_context.py b/sdks/python/apache_beam/runners/direct/evaluation_context.py index eede6c4a59c3..2b0c8644e504 100644 --- a/sdks/python/apache_beam/runners/direct/evaluation_context.py +++ b/sdks/python/apache_beam/runners/direct/evaluation_context.py @@ -354,6 +354,7 @@ def is_done(self, transform=None): def _is_transform_done(self, transform): tw = self._watermark_manager.get_watermarks(transform) + print '[!!] TRANSFORM_DONE?', transform, 'OWM', tw.output_watermark, tw.output_watermark == WatermarkManager.WATERMARK_POS_INF return tw.output_watermark == WatermarkManager.WATERMARK_POS_INF def get_value_or_schedule_after_output(self, side_input, task): diff --git a/sdks/python/apache_beam/runners/direct/executor.py b/sdks/python/apache_beam/runners/direct/executor.py index 9cd1a3311689..e33c65f7d810 100644 --- a/sdks/python/apache_beam/runners/direct/executor.py +++ b/sdks/python/apache_beam/runners/direct/executor.py @@ -577,6 +577,7 @@ def _fire_timers(self): if timer_firing is None: # Use an empty committed bundle. just to trigger. # TODO: are there still timer-using things that don't take keyed work items? + print '%%%%%%%%%%% EMPTY BUNDLE', applied_ptransform empty_bundle = ( self._executor.evaluation_context.create_empty_committed_bundle( applied_ptransform.inputs[0])) diff --git a/sdks/python/apache_beam/runners/direct/watermark_manager.py b/sdks/python/apache_beam/runners/direct/watermark_manager.py index eb7f61859f15..4312a27d9860 100644 --- a/sdks/python/apache_beam/runners/direct/watermark_manager.py +++ b/sdks/python/apache_beam/runners/direct/watermark_manager.py @@ -39,6 +39,9 @@ def __init__(self, clock, root_transforms, value_to_consumers, transform_keyed_s self._clock = clock # processing time clock self._value_to_consumers = value_to_consumers self._root_transforms = root_transforms + import pprint + print '1!!!!!!!!!!!!!!!!!!!!!!! VALUE_TO_CONSUMERS' + pprint.pprint(value_to_consumers) self._transform_keyed_states = transform_keyed_states # AppliedPTransform -> TransformWatermarks self._transform_to_watermarks = {} @@ -94,6 +97,7 @@ def get_watermarks(self, applied_ptransform): def update_watermarks(self, completed_committed_bundle, unprocessed_bundle, applied_ptransform, timer_update, outputs, earliest_hold): + print '*******UPDATE_WATERMARKS' assert isinstance(applied_ptransform, pipeline.AppliedPTransform) self._update_pending( completed_committed_bundle, unprocessed_bundle, applied_ptransform, @@ -128,6 +132,7 @@ def _update_pending(self, input_committed_bundle, unprocessed_bundle, completed_tw.remove_pending(input_committed_bundle) def _refresh_watermarks(self, applied_ptransform): + print 'RW_RECURSIVE', applied_ptransform assert isinstance(applied_ptransform, pipeline.AppliedPTransform) tw = self.get_watermarks(applied_ptransform) if tw.refresh(): @@ -143,11 +148,15 @@ def _refresh_watermarks(self, applied_ptransform): self._refresh_watermarks(consumer) def extract_fired_timers(self): + print 'EXTRACT TIMERS!!!' all_timers = [] for applied_ptransform, tw in self._transform_to_watermarks.iteritems(): + print '[!] ET', applied_ptransform, tw._output_watermark, tw._input_watermark + print ' CHILDREN: ', applied_ptransform.outputs fired_timers = tw.extract_fired_timers() if fired_timers: all_timers.append((applied_ptransform, fired_timers)) + print 'EXTRACT TIMERS END!!!', all_timers return all_timers @@ -207,6 +216,7 @@ def remove_pending(self, completed): self._pending.remove(completed) def refresh(self): + print '[!] WATERMARK REFRESH', self._label, '(existing watermark', self._output_watermark # TODO: remove this and the below assert from apache_beam.runners.direct.evaluation_context import DirectUnmergedState with self._lock: @@ -234,6 +244,7 @@ def refresh(self): self._input_watermark = max(self._input_watermark, min(pending_holder, producer_watermark, earliest_watermark_hold)) new_output_watermark = min(self._input_watermark, self._earliest_hold) + print '[!] ', self._label, 'INPUT', self._input_watermark, 'OUTPUT', new_output_watermark advanced = new_output_watermark > self._output_watermark if advanced: From 7be85c67c5c405c16c146e5605dd2a5bb9bd3bdd Mon Sep 17 00:00:00 2001 From: Charles Chen Date: Fri, 26 May 2017 15:56:32 -0700 Subject: [PATCH 15/21] Don't hold input watermark on pending elements. --- sdks/python/apache_beam/runners/direct/watermark_manager.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/direct/watermark_manager.py b/sdks/python/apache_beam/runners/direct/watermark_manager.py index 4312a27d9860..2bacb4f26b85 100644 --- a/sdks/python/apache_beam/runners/direct/watermark_manager.py +++ b/sdks/python/apache_beam/runners/direct/watermark_manager.py @@ -243,8 +243,12 @@ def refresh(self): self._input_watermark = max(self._input_watermark, min(pending_holder, producer_watermark, earliest_watermark_hold)) - new_output_watermark = min(self._input_watermark, self._earliest_hold) + # TODO: clean this up. + middle_watermark_thing = max(self._input_watermark, + min(producer_watermark, earliest_watermark_hold)) + new_output_watermark = min(middle_watermark_thing, self._earliest_hold) print '[!] ', self._label, 'INPUT', self._input_watermark, 'OUTPUT', new_output_watermark + print ' input watermark details: pending_holder', pending_holder, 'producer_watermark', producer_watermark, 'earliest_watermark_hold', earliest_watermark_hold advanced = new_output_watermark > self._output_watermark if advanced: From 1df07a5b46811d0d3a383b156121d77f5f37aa57 Mon Sep 17 00:00:00 2001 From: Charles Chen Date: Tue, 30 May 2017 11:26:03 -0700 Subject: [PATCH 16/21] Finish moving earliest hold extraction to evaluators, dbg output++ --- .../runners/direct/evaluation_context.py | 2 +- .../runners/direct/transform_evaluator.py | 10 ++++++--- .../runners/direct/watermark_manager.py | 21 ++++++++++--------- 3 files changed, 19 insertions(+), 14 deletions(-) diff --git a/sdks/python/apache_beam/runners/direct/evaluation_context.py b/sdks/python/apache_beam/runners/direct/evaluation_context.py index 2b0c8644e504..f9d43d9ee9a4 100644 --- a/sdks/python/apache_beam/runners/direct/evaluation_context.py +++ b/sdks/python/apache_beam/runners/direct/evaluation_context.py @@ -354,7 +354,7 @@ def is_done(self, transform=None): def _is_transform_done(self, transform): tw = self._watermark_manager.get_watermarks(transform) - print '[!!] TRANSFORM_DONE?', transform, 'OWM', tw.output_watermark, tw.output_watermark == WatermarkManager.WATERMARK_POS_INF + print '[!!] TRANSFORM_DONE?', transform, 'IWM', tw.input_watermark,'OWM', tw.output_watermark, tw.output_watermark == WatermarkManager.WATERMARK_POS_INF, tw._pending return tw.output_watermark == WatermarkManager.WATERMARK_POS_INF def get_value_or_schedule_after_output(self, side_input, task): diff --git a/sdks/python/apache_beam/runners/direct/transform_evaluator.py b/sdks/python/apache_beam/runners/direct/transform_evaluator.py index 023227948ec9..2661dff25b67 100644 --- a/sdks/python/apache_beam/runners/direct/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/direct/transform_evaluator.py @@ -57,10 +57,12 @@ def get_root_bundles(self): class DefaultRootBundleProvider(RootBundleProvider): """Provides an empty bundle by default for root transforms.""" - def get_root_bundles(self, ): + def get_root_bundles(self): + print 'WTF', self._applied_ptransform, self._applied_ptransform.inputs + # TODO: these are root bundles, they don't really need a producer... but apparently some inputs are not defined. empty_bundle = ( self._evaluation_context.create_empty_committed_bundle( - self._applied_ptransform.inputs[0])) + self._applied_ptransform.inputs[0] if self._applied_ptransform.inputs else pvalue.PBegin(self._applied_ptransform.transform.pipeline))) return [empty_bundle] class _TestStreamRootBundleProvider(RootBundleProvider): @@ -405,6 +407,7 @@ class _ParDoEvaluator(_TransformEvaluator): """TransformEvaluator for ParDo transform.""" def start_bundle(self): transform = self._applied_ptransform.transform + print 'TRANSFORMSI', transform, 'SIDE INPUTS', self._side_inputs self._tagged_receivers = _TaggedReceivers(self._evaluation_context) for output_tag in self._applied_ptransform.outputs: @@ -584,8 +587,9 @@ def finish_bundle(self): bundles = [] if self.gabw_items: qbundles = [bundle] + hold = self.state.get_earliest_hold() return TransformResult( - self._applied_ptransform, bundles, [], self.state, None, None, None, None) + self._applied_ptransform, bundles, [], self.state, None, None, None, hold) class _NativeWriteEvaluator(_TransformEvaluator): diff --git a/sdks/python/apache_beam/runners/direct/watermark_manager.py b/sdks/python/apache_beam/runners/direct/watermark_manager.py index 2bacb4f26b85..6c6c5a80a8c3 100644 --- a/sdks/python/apache_beam/runners/direct/watermark_manager.py +++ b/sdks/python/apache_beam/runners/direct/watermark_manager.py @@ -154,6 +154,7 @@ def extract_fired_timers(self): print '[!] ET', applied_ptransform, tw._output_watermark, tw._input_watermark print ' CHILDREN: ', applied_ptransform.outputs fired_timers = tw.extract_fired_timers() + print ' ', fired_timers if fired_timers: all_timers.append((applied_ptransform, fired_timers)) print 'EXTRACT TIMERS END!!!', all_timers @@ -228,12 +229,12 @@ def refresh(self): if bundle_min_timestamp < pending_holder: pending_holder = bundle_min_timestamp - earliest_watermark_hold = WatermarkManager.WATERMARK_POS_INF - for unused_key, state in self._keyed_states.iteritems(): - assert isinstance(state, DirectUnmergedState), state - print '~~~~~~~~WHSTATE [key=', unused_key, ']', state - earliest_watermark_hold = state.get_earliest_hold() - print 'holds [current watermark =', self._input_watermark, ']:', state.get_earliest_hold() + # earliest_watermark_hold = WatermarkManager.WATERMARK_POS_INF + # for unused_key, state in self._keyed_states.iteritems(): + # assert isinstance(state, DirectUnmergedState), state + # print '~~~~~~~~WHSTATE [key=', unused_key, ']', state + # earliest_watermark_hold = state.get_earliest_hold() + # print 'holds [current watermark =', self._input_watermark, ']:', state.get_earliest_hold() input_watermarks = [ @@ -242,13 +243,13 @@ def refresh(self): producer_watermark = min(input_watermarks) self._input_watermark = max(self._input_watermark, - min(pending_holder, producer_watermark, earliest_watermark_hold)) + min(pending_holder, producer_watermark)) # TODO: clean this up. middle_watermark_thing = max(self._input_watermark, - min(producer_watermark, earliest_watermark_hold)) + min(pending_holder, producer_watermark)) new_output_watermark = min(middle_watermark_thing, self._earliest_hold) print '[!] ', self._label, 'INPUT', self._input_watermark, 'OUTPUT', new_output_watermark - print ' input watermark details: pending_holder', pending_holder, 'producer_watermark', producer_watermark, 'earliest_watermark_hold', earliest_watermark_hold + print ' input watermark details: pending_holder', pending_holder, 'producer_watermark', producer_watermark advanced = new_output_watermark > self._output_watermark if advanced: @@ -267,7 +268,7 @@ def extract_fired_timers(self): if self._fired_timers: print 'FIRED_TIMERS? FALSE', self # Wait until fired timers have been processed. - # return False + return False fired_timers = [] From e52ba42a9efe0961ed57da52e93fdd8e5970eedc Mon Sep 17 00:00:00 2001 From: Charles Chen Date: Wed, 31 May 2017 15:46:06 -0700 Subject: [PATCH 17/21] Batch WordCount works in streaming mode!! Fixes earliest hold to be < instead of <=. --- .../consumer_tracking_pipeline_visitor.py | 2 ++ .../runners/direct/evaluation_context.py | 29 +++++++++++++++++-- .../apache_beam/runners/direct/executor.py | 2 ++ .../runners/direct/transform_evaluator.py | 5 ++-- .../runners/direct/transform_result.py | 1 + .../runners/direct/watermark_manager.py | 25 +++++++++------- sdks/python/apache_beam/transforms/core.py | 2 +- sdks/python/apache_beam/transforms/trigger.py | 2 +- 8 files changed, 51 insertions(+), 17 deletions(-) diff --git a/sdks/python/apache_beam/runners/direct/consumer_tracking_pipeline_visitor.py b/sdks/python/apache_beam/runners/direct/consumer_tracking_pipeline_visitor.py index d625d3ce5cee..03583b713b17 100644 --- a/sdks/python/apache_beam/runners/direct/consumer_tracking_pipeline_visitor.py +++ b/sdks/python/apache_beam/runners/direct/consumer_tracking_pipeline_visitor.py @@ -42,6 +42,7 @@ def __init__(self): self._num_transforms = 0 def visit_transform(self, applied_ptransform): + print 'VISIT', applied_ptransform, '; inputs:', applied_ptransform.inputs, '; parent:', applied_ptransform.parent inputs = list(applied_ptransform.inputs) if inputs: for input_value in inputs: @@ -55,4 +56,5 @@ def visit_transform(self, applied_ptransform): self.step_names[applied_ptransform] = 's%d' % (self._num_transforms) self._num_transforms += 1 for side_input in applied_ptransform.side_inputs: + print 'VIEW', side_input self.views.append(side_input) diff --git a/sdks/python/apache_beam/runners/direct/evaluation_context.py b/sdks/python/apache_beam/runners/direct/evaluation_context.py index f9d43d9ee9a4..b8a2260757e3 100644 --- a/sdks/python/apache_beam/runners/direct/evaluation_context.py +++ b/sdks/python/apache_beam/runners/direct/evaluation_context.py @@ -117,16 +117,20 @@ def get_value_or_schedule_after_output(self, side_input, task): def add_values(self, side_input, values): with self._lock: view = self._views[side_input] + if view.has_result: + print 'ERROR!!! VIEW ALREADY HAS RESULT', side_input, values assert not view.has_result view.elements.extend(values) def finalize_value_and_get_tasks(self, side_input): + print '***** FINALIZE VIEWWWWW', side_input with self._lock: view = self._views[side_input] assert not view.has_result assert view.value is None assert view.callable_queue is not None view.value = self._pvalue_to_value(side_input, view.elements) + print 'result', view.elements, view.value view.elements = None result = tuple(view.callable_queue) for task in result: @@ -183,6 +187,9 @@ def __init__(self, pipeline_options, bundle_factory, root_transforms, self._pcollection_to_views = collections.defaultdict(list) for view in views: self._pcollection_to_views[view.pvalue].append(view) + import pprint + print 'PCOLLECTION TO VIEWS' + pprint.pprint(self._pcollection_to_views) # AppliedPTransform -> Evaluator specific state objects self._legacy_existing_state = {} @@ -258,22 +265,38 @@ def handle_result( self._metrics.commit_logical(completed_bundle, result.logical_metric_updates) + print 'HANDLE RESULT COMMITED', committed_bundles, result.uncommitted_output_bundles, result.unprocessed_bundle # If the result is for a view, update side inputs container. if (result.uncommitted_output_bundles and result.uncommitted_output_bundles[0].pcollection in self._pcollection_to_views): + print '***** HI I AM A VIEW', result.uncommitted_output_bundles[0].pcollection, committed_bundles for view in self._pcollection_to_views[ result.uncommitted_output_bundles[0].pcollection]: for committed_bundle in committed_bundles: + print 'ADD TO VIEW', committed_bundle.get_elements_iterable(make_copy=True) # side_input must be materialized. self._side_inputs_container.add_values( view, committed_bundle.get_elements_iterable(make_copy=True)) - if (self.get_execution_context(result.transform, None) # TODO(ccy): what is the key here? - .watermarks.input_watermark - == WatermarkManager.WATERMARK_POS_INF): + completed = True + print 'STUFF', self._transform_keyed_states[result.transform] + watermarks = self._watermark_manager.get_watermarks(result.transform) + if watermarks._pending: + completed = False + print 'WATERMARKS', watermarks, watermarks._pending + for key in self._transform_keyed_states[result.transform]: + if (self.get_execution_context(result.transform, key).watermarks.input_watermark + < WatermarkManager.WATERMARK_POS_INF or + self.get_execution_context(result.transform, key).watermarks._pending): + print '&&& COMPLETED FALSE', result.transform, key, self.get_execution_context(result.transform, key).watermarks._pending + completed = False + if completed: + print '***** HI I AM COMPLETED', result.uncommitted_output_bundles[0].pcollection self._pending_unblocked_tasks.extend( self._side_inputs_container.finalize_value_and_get_tasks(view)) + else: + print '***** HI I AM *NOT* COMPLETED', result.uncommitted_output_bundles[0].pcollection if result.counters: for counter in result.counters: diff --git a/sdks/python/apache_beam/runners/direct/executor.py b/sdks/python/apache_beam/runners/direct/executor.py index e33c65f7d810..55e53741c206 100644 --- a/sdks/python/apache_beam/runners/direct/executor.py +++ b/sdks/python/apache_beam/runners/direct/executor.py @@ -231,6 +231,7 @@ def __init__(self, evaluation_context, all_updates, timers=None): def handle_result(self, transform_executor, input_committed_bundle, transform_result): output_committed_bundles = self._evaluation_context.handle_result( input_committed_bundle, self._timers, transform_result) + print 'COMPLETIONCALLBACK HANDLE RESULT', output_committed_bundles for output_committed_bundle in output_committed_bundles: self._all_updates.offer( _ExecutorServiceParallelExecutor._ExecutorUpdate( @@ -285,6 +286,7 @@ def call(self): self._evaluation_context.get_value_or_schedule_after_output( side_input, self)) if not has_result: + print '$$$$$$ CALL TO', self, 'DELAYED BASED ON UNAVAILABLE SIDE INPUT', side_input # Monitor task will reschedule this executor once the side input is # available. return diff --git a/sdks/python/apache_beam/runners/direct/transform_evaluator.py b/sdks/python/apache_beam/runners/direct/transform_evaluator.py index 2661dff25b67..d00cccf136e0 100644 --- a/sdks/python/apache_beam/runners/direct/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/direct/transform_evaluator.py @@ -553,6 +553,7 @@ def start_bundle(self): def process_element(self, element): kwi = element.value print '[!] GABW process_element', element, kwi + assert isinstance(kwi, KeyedWorkItem), (element, kwi) k, timers, vs = kwi.key, kwi.timers, kwi.elements encoded_k = self.key_coder.encode(k) state = self.state @@ -581,12 +582,12 @@ def process_element(self, element): def finish_bundle(self): print '[!] GABW_OUTPUT', self.gabw_items bundle = self._evaluation_context.create_bundle( - self._applied_ptransform.inputs[0]) + self.output_pcollection) for item in self.gabw_items: # TODO bundle.add(item) bundles = [] if self.gabw_items: - qbundles = [bundle] + bundles = [bundle] hold = self.state.get_earliest_hold() return TransformResult( self._applied_ptransform, bundles, [], self.state, None, None, None, hold) diff --git a/sdks/python/apache_beam/runners/direct/transform_result.py b/sdks/python/apache_beam/runners/direct/transform_result.py index 153581478d42..f6f527af2c6a 100644 --- a/sdks/python/apache_beam/runners/direct/transform_result.py +++ b/sdks/python/apache_beam/runners/direct/transform_result.py @@ -29,6 +29,7 @@ def __init__(self, applied_ptransform, uncommitted_output_bundles, unprocessed_bundle, state, legacy_state, timer_update, counters, watermark_hold, undeclared_tag_values=None): + print '^^^ NEW TR', applied_ptransform, uncommitted_output_bundles, unprocessed_bundle self.transform = applied_ptransform self.uncommitted_output_bundles = uncommitted_output_bundles self.unprocessed_bundle = unprocessed_bundle diff --git a/sdks/python/apache_beam/runners/direct/watermark_manager.py b/sdks/python/apache_beam/runners/direct/watermark_manager.py index 6c6c5a80a8c3..e1485e003f0c 100644 --- a/sdks/python/apache_beam/runners/direct/watermark_manager.py +++ b/sdks/python/apache_beam/runners/direct/watermark_manager.py @@ -222,12 +222,17 @@ def refresh(self): from apache_beam.runners.direct.evaluation_context import DirectUnmergedState with self._lock: pending_holder = WatermarkManager.WATERMARK_POS_INF + has_pending = False + print '[!!] PENDING ELEMENTS:', self._pending for input_bundle in self._pending: # TODO: Perhaps we can have the Bundle class keep track of the minimum # timestamp so we don't have to do an iteration here. bundle_min_timestamp = min(wv.timestamp for wv in input_bundle.get_elements_iterable()) if bundle_min_timestamp < pending_holder: pending_holder = bundle_min_timestamp + has_pending = True + if self._pending: + pending_holder = WatermarkManager.WATERMARK_NEG_INF # earliest_watermark_hold = WatermarkManager.WATERMARK_POS_INF # for unused_key, state in self._keyed_states.iteritems(): @@ -244,10 +249,10 @@ def refresh(self): self._input_watermark = max(self._input_watermark, min(pending_holder, producer_watermark)) - # TODO: clean this up. - middle_watermark_thing = max(self._input_watermark, - min(pending_holder, producer_watermark)) - new_output_watermark = min(middle_watermark_thing, self._earliest_hold) + # # TODO: clean this up. + # middle_watermark_thing = max(self._input_watermark, + # min(pending_holder, producer_watermark)) + new_output_watermark = min(self._input_watermark, self._earliest_hold) print '[!] ', self._label, 'INPUT', self._input_watermark, 'OUTPUT', new_output_watermark print ' input watermark details: pending_holder', pending_holder, 'producer_watermark', producer_watermark @@ -282,12 +287,12 @@ def extract_fired_timers(self): fired_timers.append(TimerFiring(key, time_domain, timestamp)) - # TODO: check if this hack is still necessary. - should_fire = ( - self._earliest_hold < WatermarkManager.WATERMARK_POS_INF and - self._input_watermark == WatermarkManager.WATERMARK_POS_INF) - if should_fire: - fired_timers.append(None) # Sentinel representing legacy timer firing. + # # TODO: check if this hack is still necessary. + # should_fire = ( + # self._earliest_hold < WatermarkManager.WATERMARK_POS_INF and + # self._input_watermark == WatermarkManager.WATERMARK_POS_INF) + # if should_fire: + # fired_timers.append(None) # Sentinel representing legacy timer firing. self._fired_timers = fired_timers return fired_timers diff --git a/sdks/python/apache_beam/transforms/core.py b/sdks/python/apache_beam/transforms/core.py index b4919201deb7..40125389e220 100644 --- a/sdks/python/apache_beam/transforms/core.py +++ b/sdks/python/apache_beam/transforms/core.py @@ -1446,7 +1446,7 @@ def expand(self, pbegin): self.infer_output_type(None)) coder = typecoders.registry.get_coder(ouput_type) source = self._create_source_from_iterable(self.value, coder) - return pbegin.pipeline | iobase.Read(source).with_output_types(ouput_type) + return pbegin.pipeline | "Create" >> iobase.Read(source).with_output_types(ouput_type) def get_windowing(self, unused_inputs): return Windowing(GlobalWindows()) diff --git a/sdks/python/apache_beam/transforms/trigger.py b/sdks/python/apache_beam/transforms/trigger.py index 42082c3bf387..b2acb3029294 100644 --- a/sdks/python/apache_beam/transforms/trigger.py +++ b/sdks/python/apache_beam/transforms/trigger.py @@ -1125,7 +1125,7 @@ def get_earliest_hold(self): for window, tagged_states in self.state.iteritems(): # TODO: is this general enough? if 'watermark' in tagged_states: - hold = min(tagged_states['watermark']) + hold = min(tagged_states['watermark']) - 1 # TODO: THIS SHOULD ACTUALLY BE 1 MICRO IF POSSIBLE earliest_hold = min(earliest_hold, hold) return earliest_hold From e5bb716e00ac02679edc3a357153009b862bb4d0 Mon Sep 17 00:00:00 2001 From: Charles Chen Date: Mon, 5 Jun 2017 15:20:37 -0700 Subject: [PATCH 18/21] Working batch and streaming mode execution, except for certain empty side inputs. --- sdks/python/apache_beam/runners/direct/executor.py | 2 ++ .../apache_beam/runners/direct/streaming_test.py | 3 ++- .../apache_beam/runners/direct/transform_evaluator.py | 11 ++++++++--- .../apache_beam/runners/direct/watermark_manager.py | 7 ++++--- sdks/python/apache_beam/transforms/core.py | 6 ++++++ sdks/python/apache_beam/transforms/trigger.py | 4 +++- sdks/python/apache_beam/utils/timestamp.py | 3 +++ 7 files changed, 28 insertions(+), 8 deletions(-) diff --git a/sdks/python/apache_beam/runners/direct/executor.py b/sdks/python/apache_beam/runners/direct/executor.py index 55e53741c206..56c370b2e74d 100644 --- a/sdks/python/apache_beam/runners/direct/executor.py +++ b/sdks/python/apache_beam/runners/direct/executor.py @@ -312,7 +312,9 @@ def call(self): result = evaluator.finish_bundle() result.logical_metric_updates = metrics_container.get_cumulative() + print 'self._evaluation_context.has_cache', self._evaluation_context.has_cache if self._evaluation_context.has_cache: + print 'result.uncommitted_output_bundles', result.uncommitted_output_bundles for uncommitted_bundle in result.uncommitted_output_bundles: self._evaluation_context.append_to_cache( self._applied_ptransform, uncommitted_bundle.tag, diff --git a/sdks/python/apache_beam/runners/direct/streaming_test.py b/sdks/python/apache_beam/runners/direct/streaming_test.py index 8b2f88c3fd26..ceb166602f25 100644 --- a/sdks/python/apache_beam/runners/direct/streaming_test.py +++ b/sdks/python/apache_beam/runners/direct/streaming_test.py @@ -35,7 +35,8 @@ def test_test_stream(self): .add_elements(['e']) .advance_processing_time(10) .advance_watermark_to(300) - .add_elements([window.TimestampedValue('late', 12)])) + .add_elements([window.TimestampedValue('late', 12)]) + .add_elements([window.TimestampedValue('last', 310)])) p = beam.Pipeline(runner=DirectRunner()) elements = p | test_stream groups = elements | beam.Map(lambda x: ('k', x))| beam.WindowInto(window.FixedWindows(15)) | beam.GroupByKey() diff --git a/sdks/python/apache_beam/runners/direct/transform_evaluator.py b/sdks/python/apache_beam/runners/direct/transform_evaluator.py index d00cccf136e0..2669c2c4716d 100644 --- a/sdks/python/apache_beam/runners/direct/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/direct/transform_evaluator.py @@ -33,6 +33,7 @@ from apache_beam.runners.dataflow.native_io.iobase import _NativeWrite # pylint: disable=protected-access from apache_beam.testing.test_stream import TestStream from apache_beam.transforms import core +from apache_beam.utils.timestamp import MAX_TIMESTAMP from apache_beam.utils.timestamp import MIN_TIMESTAMP from apache_beam.transforms.trigger import create_trigger_driver from apache_beam.transforms.window import GlobalWindows @@ -321,10 +322,12 @@ def finish_bundle(self): raise ValueError('Invalid TestStream event: %s.' % event) unprocessed_bundle = None + hold = MAX_TIMESTAMP if self.current_index < len(self.test_stream.events) - 1: unprocessed_bundle = self._evaluation_context.create_bundle( self._applied_ptransform.inputs[0]) unprocessed_bundle.add(GlobalWindows.windowed_value(self.current_index + 1, timestamp=watermark)) + hold = watermark @@ -332,7 +335,7 @@ def finish_bundle(self): return TransformResult( self._applied_ptransform, bundles, unprocessed_bundle, None, None, None, - None, None) + None, hold) class _FlattenEvaluator(_TransformEvaluator): @@ -477,7 +480,6 @@ def start_bundle(self): assert len(self._outputs) == 1 self.output_pcollection = list(self._outputs)[0] - # The input type of a GroupByKey will be KV[Any, Any] or more specific. kv_type_hint = ( self._applied_ptransform.transform.get_type_hints().input_types[0]) self.key_coder = coders.registry.get_coder(kv_type_hint[0].tuple_types[0]) @@ -502,7 +504,9 @@ def finish_bundle(self): kwi = KeyedWorkItem(k, elements=vs) bundle.add(GlobalWindows.windowed_value(kwi)) bundles.append(bundle) - + # if not bundles: + # bundle = self._evaluation_context.create_keyed_bundle(self.output_pcollection, None) + # bundles.append(bundle) return TransformResult( self._applied_ptransform, bundles, [], None, None, None, None, None) @@ -545,6 +549,7 @@ def start_bundle(self): self.gabw_items = [] # The input type of a GroupByKey will be KV[Any, Any] or more specific. + print 'GABW_TRANSFORM', self._applied_ptransform.transform, self._applied_ptransform.transform.get_type_hints(), self._applied_ptransform.transform.get_type_hints().input_types kv_type_hint = ( self._applied_ptransform.transform.get_type_hints().input_types[0]) self.key_coder = coders.registry.get_coder(kv_type_hint[0].tuple_types[0]) diff --git a/sdks/python/apache_beam/runners/direct/watermark_manager.py b/sdks/python/apache_beam/runners/direct/watermark_manager.py index e1485e003f0c..42214dcca1a7 100644 --- a/sdks/python/apache_beam/runners/direct/watermark_manager.py +++ b/sdks/python/apache_beam/runners/direct/watermark_manager.py @@ -25,6 +25,7 @@ from apache_beam import pvalue from apache_beam.utils.timestamp import MAX_TIMESTAMP from apache_beam.utils.timestamp import MIN_TIMESTAMP +from apache_beam.utils.timestamp import TIME_GRANULARITY class WatermarkManager(object): @@ -227,12 +228,12 @@ def refresh(self): for input_bundle in self._pending: # TODO: Perhaps we can have the Bundle class keep track of the minimum # timestamp so we don't have to do an iteration here. - bundle_min_timestamp = min(wv.timestamp for wv in input_bundle.get_elements_iterable()) + bundle_min_timestamp = min(wv.timestamp for wv in input_bundle.get_elements_iterable()) - TIME_GRANULARITY if bundle_min_timestamp < pending_holder: pending_holder = bundle_min_timestamp has_pending = True - if self._pending: - pending_holder = WatermarkManager.WATERMARK_NEG_INF + # if self._pending: + # pending_holder = WatermarkManager.WATERMARK_NEG_INF # earliest_watermark_hold = WatermarkManager.WATERMARK_POS_INF # for unused_key, state in self._keyed_states.iteritems(): diff --git a/sdks/python/apache_beam/transforms/core.py b/sdks/python/apache_beam/transforms/core.py index 40125389e220..484fa023fd62 100644 --- a/sdks/python/apache_beam/transforms/core.py +++ b/sdks/python/apache_beam/transforms/core.py @@ -1125,6 +1125,8 @@ def expand(self, pcoll): return pvalue.PCollection(pcoll.pipeline) +@typehints.with_input_types(typehints.KV[K, typehints.Iterable[V]]) +@typehints.with_output_types(typehints.KV[K, typehints.Iterable[V]]) class _GroupAlsoByWindow(ParDo): """The GroupAlsoByWindow transform.""" @@ -1133,6 +1135,10 @@ def __init__(self, windowing): _GroupAlsoByWindowDoFn(windowing)) self.windowing = windowing + def expand(self, pcoll): + self._check_pcollection(pcoll) + return pvalue.PCollection(pcoll.pipeline) + class _GroupAlsoByWindowDoFn(DoFn): # TODO(robertwb): Support combiner lifting. diff --git a/sdks/python/apache_beam/transforms/trigger.py b/sdks/python/apache_beam/transforms/trigger.py index b2acb3029294..1730d95a9d1e 100644 --- a/sdks/python/apache_beam/transforms/trigger.py +++ b/sdks/python/apache_beam/transforms/trigger.py @@ -36,6 +36,7 @@ from apache_beam.runners.api import beam_runner_api_pb2 from apache_beam.utils.timestamp import MAX_TIMESTAMP from apache_beam.utils.timestamp import MIN_TIMESTAMP +from apache_beam.utils.timestamp import TIME_GRANULARITY # AfterCount is experimental. No backwards compatibility guarantees. @@ -836,6 +837,7 @@ def create_trigger_driver(windowing, is_batch=False, phased_combine_fn=None): driver = DefaultGlobalBatchTriggerDriver() else: driver = GeneralTriggerDriver(windowing) + print 'DRIVER', driver if phased_combine_fn: # TODO(ccy): Refactor GeneralTriggerDriver to combine values eagerly using @@ -1125,7 +1127,7 @@ def get_earliest_hold(self): for window, tagged_states in self.state.iteritems(): # TODO: is this general enough? if 'watermark' in tagged_states: - hold = min(tagged_states['watermark']) - 1 # TODO: THIS SHOULD ACTUALLY BE 1 MICRO IF POSSIBLE + hold = min(tagged_states['watermark']) - TIME_GRANULARITY earliest_hold = min(earliest_hold, hold) return earliest_hold diff --git a/sdks/python/apache_beam/utils/timestamp.py b/sdks/python/apache_beam/utils/timestamp.py index 5d1b48c14e33..149585fdcadc 100644 --- a/sdks/python/apache_beam/utils/timestamp.py +++ b/sdks/python/apache_beam/utils/timestamp.py @@ -208,3 +208,6 @@ def __rmul__(self, other): def __mod__(self, other): other = Duration.of(other) return Duration(micros=self.micros % other.micros) + + +TIME_GRANULARITY = Duration(micros=1) From 91873cc1422d5bc7812464feea6789a1c4242577 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Wed, 7 Jun 2017 08:48:49 -0700 Subject: [PATCH 19/21] PubSubIO for direct runner. --- sdks/python/apache_beam/io/gcp/pubsub.py | 10 ++- sdks/python/apache_beam/pipeline.py | 2 +- sdks/python/apache_beam/pvalue.py | 5 +- .../runners/direct/direct_runner.py | 28 ++++++++ .../runners/direct/transform_evaluator.py | 70 ++++++++++++++++++- .../runners/direct/watermark_manager.py | 10 ++- 6 files changed, 116 insertions(+), 9 deletions(-) diff --git a/sdks/python/apache_beam/io/gcp/pubsub.py b/sdks/python/apache_beam/io/gcp/pubsub.py index 1ba8ac051272..5647d92c790d 100644 --- a/sdks/python/apache_beam/io/gcp/pubsub.py +++ b/sdks/python/apache_beam/io/gcp/pubsub.py @@ -28,6 +28,9 @@ from apache_beam.io.iobase import Read from apache_beam.io.iobase import Write from apache_beam.runners.dataflow.native_io import iobase as dataflow_io +from apache_beam.transforms import core +from apache_beam.transforms import window +from apache_beam.transforms import Map from apache_beam.transforms import PTransform from apache_beam.transforms import ParDo from apache_beam.transforms.display import DisplayDataItem @@ -60,10 +63,13 @@ def __init__(self, topic, subscription=None, id_label=None): subscription=subscription, id_label=id_label) + def get_windowing(self, unused_inputs): + return core.Windowing(window.GlobalWindows()) + def expand(self, pvalue): pcoll = pvalue.pipeline | Read(self._source) pcoll.element_type = bytes - pcoll = pcoll | 'decode string' >> ParDo(_decodeUtf8String) + pcoll = pcoll | 'decode string' >> Map(_decodeUtf8String) pcoll.element_type = unicode return pcoll @@ -81,7 +87,7 @@ def __init__(self, topic): self._sink = _PubSubPayloadSink(topic) def expand(self, pcoll): - pcoll = pcoll | 'encode string' >> ParDo(_encodeUtf8String) + pcoll = pcoll | 'encode string' >> Map(_encodeUtf8String) pcoll.element_type = bytes return pcoll | Write(self._sink) diff --git a/sdks/python/apache_beam/pipeline.py b/sdks/python/apache_beam/pipeline.py index 9093abfccfc3..dcdf19e85050 100644 --- a/sdks/python/apache_beam/pipeline.py +++ b/sdks/python/apache_beam/pipeline.py @@ -161,7 +161,7 @@ def run(self, test_runner_api=True): """Runs the pipeline. Returns whatever our runner returns after running.""" # When possible, invoke a round trip through the runner API. - if test_runner_api and self._verify_runner_api_compatible(): + if test_runner_api and self._verify_runner_api_compatible() and False: return Pipeline.from_runner_api( self.to_runner_api(), self.runner, self._options).run(False) diff --git a/sdks/python/apache_beam/pvalue.py b/sdks/python/apache_beam/pvalue.py index 7385e82c3a5c..c4f707005b3e 100644 --- a/sdks/python/apache_beam/pvalue.py +++ b/sdks/python/apache_beam/pvalue.py @@ -157,7 +157,10 @@ class PBegin(PValue): transforms. This allows us to have transforms that uniformly take PValue(s) as inputs. """ - pass + @property + def windowing(self): + from apache_beam.transforms import core, window + return core.Windowing(window.GlobalWindows()) class PDone(PValue): diff --git a/sdks/python/apache_beam/runners/direct/direct_runner.py b/sdks/python/apache_beam/runners/direct/direct_runner.py index 2a6336a71e64..26a706801b74 100644 --- a/sdks/python/apache_beam/runners/direct/direct_runner.py +++ b/sdks/python/apache_beam/runners/direct/direct_runner.py @@ -26,6 +26,7 @@ import collections import logging +import apache_beam as beam from apache_beam import pvalue from apache_beam.metrics.execution import MetricsEnvironment from apache_beam.runners.direct.bundle_factory import BundleFactory @@ -61,6 +62,33 @@ def apply_GroupAlsoByWindow(self, transform, pcoll): transform._check_pcollection(pcoll) return pvalue.PCollection(pcoll.pipeline) + def apply_ReadStringsFromPubSub(self, transform, pcoll): + # Execute this as a native transform. + return pvalue.PCollection(pcoll.pipeline) + + def apply_WriteStringsToPubSub(self, transform, pcoll): + from google.cloud import pubsub + topic_name = transform._sink.topic + class WriteToPubSub(beam.DoFn): + _topic = None + def start_bundle(self): + if self._topic is None: + self._topic = pubsub.Client().topic(topic_name) + self._buffer = [] + def process(self, elem): + self._buffer.append(elem.encode('utf-8')) + if len(self._buffer) >= 100: + self._flush() + def finish_bundle(self): + self._flush() + def _flush(self): + if self._buffer: + with self._topic.batch() as batch: + for datum in self._buffer: + batch.publish(datum) + self._buffer = [] + return pcoll | beam.ParDo(WriteToPubSub()) + def run(self, pipeline): """Execute the entire pipeline and returns an DirectPipelineResult.""" diff --git a/sdks/python/apache_beam/runners/direct/transform_evaluator.py b/sdks/python/apache_beam/runners/direct/transform_evaluator.py index 2669c2c4716d..4f9a2f54e43d 100644 --- a/sdks/python/apache_beam/runners/direct/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/direct/transform_evaluator.py @@ -20,6 +20,9 @@ from __future__ import absolute_import import collections +import functools +import random +import time from apache_beam import coders from apache_beam import pvalue @@ -37,6 +40,7 @@ from apache_beam.utils.timestamp import MIN_TIMESTAMP from apache_beam.transforms.trigger import create_trigger_driver from apache_beam.transforms.window import GlobalWindows +from apache_beam.transforms.window import Timestamp from apache_beam.transforms.window import WindowedValue from apache_beam.typehints.typecheck import OutputCheckWrapperDoFn from apache_beam.typehints.typecheck import TypeCheckError @@ -92,6 +96,7 @@ def __init__(self, evaluation_context): self._evaluation_context = evaluation_context self._evaluators = { io.Read: _BoundedReadEvaluator, + io.ReadStringsFromPubSub: _PubSubReadEvaluator, core.Flatten: _FlattenEvaluator, core.ParDo: _ParDoEvaluator, core._GroupByKeyOnly: _GroupByKeyOnlyEvaluator, @@ -297,7 +302,7 @@ def process_element(self, element): assert isinstance(index, int) assert 0 <= index <= len(self.test_stream.events) self.current_index = index - + def finish_bundle(self): assert len(self._outputs) == 1 output_pcollection = list(self._outputs)[0] @@ -316,7 +321,7 @@ def finish_bundle(self): assert event.new_watermark >= self.watermark watermark = event.new_watermark elif isinstance(event, ProcessingTimeEvent): - # TODO: advance processing time in the context's mock clock. + # TODO: advance processing time in the context's mock clock. pass else: raise ValueError('Invalid TestStream event: %s.' % event) @@ -338,6 +343,67 @@ def finish_bundle(self): None, hold) +class _PubSubReadEvaluator(_TransformEvaluator): + """TransformEvaluator for PubSub read.""" + + def __init__(self, *args, **kwargs): + super(_PubSubReadEvaluator, self).__init__(*args, **kwargs) + source = self._applied_ptransform.transform._source + subscription_name = ( + source.subscription or 'dataflow_%x' % random.randrange(1 << 32)) + self._subscription = self.create_subscription( + source.topic, subscription_name) + if not source.subscription: + self._subscription.create() + + _subscription_cache = {} + @classmethod + def create_subscription(cls, topic, subscription_name): + key = topic, subscription_name + if key not in cls._subscription_cache: + from google.cloud import pubsub + cls._subscription_cache[key] = ( + pubsub.Client().topic(topic).subscription(subscription_name)) + return cls._subscription_cache[key] + + def __del__(self): + if not self._applied_ptransform.transform._source.subscription: + self._subscription.delete() + + def start_bundle(self): + pass + + def process_element(self, element): + pass + + def finish_bundle(self): + data = self._read_from_pubsub() + if data: + output_pcollection = list(self._outputs)[0] + bundle = self._evaluation_context.create_bundle(output_pcollection) + now = Timestamp.of(time.time()) + for message_data in data: + bundle.output(GlobalWindows.windowed_value(message_data, timestamp=now)) + bundles = [bundle] + else: + bundles = [] + + unprocessed_bundle = self._evaluation_context.create_bundle( + self._applied_ptransform.inputs[0]) + + return TransformResult( + self._applied_ptransform, bundles, + unprocessed_bundle, None, None, None, None, + Timestamp.of(time.time())) + + def _read_from_pubsub(self): + results = self._subscription.pull(return_immediately=True) + if results: + # Direct runner has no retry. + self._subscription.acknowledge([ack_id for ack_id, message in results]) + return [message.data for ack_id, message in results] + + class _FlattenEvaluator(_TransformEvaluator): """TransformEvaluator for Flatten transform.""" diff --git a/sdks/python/apache_beam/runners/direct/watermark_manager.py b/sdks/python/apache_beam/runners/direct/watermark_manager.py index 42214dcca1a7..13bbd4c54fe8 100644 --- a/sdks/python/apache_beam/runners/direct/watermark_manager.py +++ b/sdks/python/apache_beam/runners/direct/watermark_manager.py @@ -228,9 +228,13 @@ def refresh(self): for input_bundle in self._pending: # TODO: Perhaps we can have the Bundle class keep track of the minimum # timestamp so we don't have to do an iteration here. - bundle_min_timestamp = min(wv.timestamp for wv in input_bundle.get_elements_iterable()) - TIME_GRANULARITY - if bundle_min_timestamp < pending_holder: - pending_holder = bundle_min_timestamp + try: + bundle_min_timestamp = min(wv.timestamp for wv in input_bundle.get_elements_iterable()) - TIME_GRANULARITY + if bundle_min_timestamp < pending_holder: + pending_holder = bundle_min_timestamp + except ValueError: + # Thrown by min when bundle is empty. + pass has_pending = True # if self._pending: # pending_holder = WatermarkManager.WATERMARK_NEG_INF From 12a95178163ef3bf56f49f6276194e2ab0068999 Mon Sep 17 00:00:00 2001 From: Charles Chen Date: Thu, 8 Jun 2017 15:12:14 -0700 Subject: [PATCH 20/21] Use keyed holds in watermark manager --- .../runners/direct/watermark_manager.py | 22 +++++++++++++------ 1 file changed, 15 insertions(+), 7 deletions(-) diff --git a/sdks/python/apache_beam/runners/direct/watermark_manager.py b/sdks/python/apache_beam/runners/direct/watermark_manager.py index 13bbd4c54fe8..6b07d337f4de 100644 --- a/sdks/python/apache_beam/runners/direct/watermark_manager.py +++ b/sdks/python/apache_beam/runners/direct/watermark_manager.py @@ -104,7 +104,7 @@ def update_watermarks(self, completed_committed_bundle, unprocessed_bundle, completed_committed_bundle, unprocessed_bundle, applied_ptransform, timer_update, outputs) tw = self.get_watermarks(applied_ptransform) - tw.hold(earliest_hold) + tw.hold(completed_committed_bundle.key, earliest_hold) self._refresh_watermarks(applied_ptransform) def _update_pending(self, input_committed_bundle, unprocessed_bundle, @@ -171,7 +171,9 @@ def __init__(self, clock, keyed_states, label=None): self._input_transform_watermarks = [] self._input_watermark = WatermarkManager.WATERMARK_NEG_INF self._output_watermark = WatermarkManager.WATERMARK_NEG_INF - self._earliest_hold = WatermarkManager.WATERMARK_POS_INF + # TODO(ccy): in the future, we should integrate this with holds in the + # Timer / State API. + self._keyed_earliest_holds = {} self._pending = set() # Scheduled bundles targeted for this transform. self._fired_timers = False self._lock = threading.Lock() @@ -200,11 +202,13 @@ def output_watermark(self): with self._lock: return self._output_watermark - def hold(self, value): + def hold(self, key, value): with self._lock: - if value is None: - value = WatermarkManager.WATERMARK_POS_INF - self._earliest_hold = value + if value is None or value == WatermarkManager.WATERMARK_POS_INF: + if key in self._keyed_earliest_holds: + del self._keyed_earliest_holds[key] + else: + self._keyed_earliest_holds[key] = value def add_pending(self, pending): with self._lock: @@ -245,6 +249,10 @@ def refresh(self): # print '~~~~~~~~WHSTATE [key=', unused_key, ']', state # earliest_watermark_hold = state.get_earliest_hold() # print 'holds [current watermark =', self._input_watermark, ']:', state.get_earliest_hold() + earliest_hold = WatermarkManager.WATERMARK_POS_INF + for unused_key, hold in self._keyed_earliest_holds.iteritems(): + if hold - TIME_GRANULARITY < earliest_hold: + earliest_hold = hold - TIME_GRANULARITY input_watermarks = [ @@ -257,7 +265,7 @@ def refresh(self): # # TODO: clean this up. # middle_watermark_thing = max(self._input_watermark, # min(pending_holder, producer_watermark)) - new_output_watermark = min(self._input_watermark, self._earliest_hold) + new_output_watermark = min(self._input_watermark, earliest_hold) print '[!] ', self._label, 'INPUT', self._input_watermark, 'OUTPUT', new_output_watermark print ' input watermark details: pending_holder', pending_holder, 'producer_watermark', producer_watermark From b90a2e45b6b39d3d6c513218a1ec3ae9e1edca27 Mon Sep 17 00:00:00 2001 From: Charles Chen Date: Fri, 9 Jun 2017 08:53:03 -0700 Subject: [PATCH 21/21] Fix dropped messages in PubSubRead. --- .../runners/direct/transform_evaluator.py | 32 +++++++++++-------- 1 file changed, 18 insertions(+), 14 deletions(-) diff --git a/sdks/python/apache_beam/runners/direct/transform_evaluator.py b/sdks/python/apache_beam/runners/direct/transform_evaluator.py index 4f9a2f54e43d..7a06a43414ab 100644 --- a/sdks/python/apache_beam/runners/direct/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/direct/transform_evaluator.py @@ -351,10 +351,12 @@ def __init__(self, *args, **kwargs): source = self._applied_ptransform.transform._source subscription_name = ( source.subscription or 'dataflow_%x' % random.randrange(1 << 32)) - self._subscription = self.create_subscription( - source.topic, subscription_name) - if not source.subscription: - self._subscription.create() + self._subscription = self._execution_context.legacy_existing_state + if not self._subscription: + self._subscription = self.create_subscription( + source.topic, subscription_name) + if not source.subscription: + self._subscription.create() _subscription_cache = {} @classmethod @@ -363,12 +365,14 @@ def create_subscription(cls, topic, subscription_name): if key not in cls._subscription_cache: from google.cloud import pubsub cls._subscription_cache[key] = ( - pubsub.Client().topic(topic).subscription(subscription_name)) + pubsub.Client().topic(topic).subscription(subscription_name, ack_deadline=30)) return cls._subscription_cache[key] - def __del__(self): - if not self._applied_ptransform.transform._source.subscription: - self._subscription.delete() + # TODO(ccy): this should be run when we terminate execution, but not when + # this specific evaluator instance is deleted. + # def __del__(self): + # if not self._applied_ptransform.transform._source.subscription: + # self._subscription.delete() def start_bundle(self): pass @@ -393,15 +397,15 @@ def finish_bundle(self): return TransformResult( self._applied_ptransform, bundles, - unprocessed_bundle, None, None, None, None, + unprocessed_bundle, None, self._subscription, None, None, Timestamp.of(time.time())) def _read_from_pubsub(self): - results = self._subscription.pull(return_immediately=True) - if results: - # Direct runner has no retry. - self._subscription.acknowledge([ack_id for ack_id, message in results]) - return [message.data for ack_id, message in results] + from google.cloud import pubsub + print 'READ FROM PUBSUB' + with pubsub.subscription.AutoAck(self._subscription, return_immediately=True, max_messages=10) as results: + print 'DONE READ FROM PUBSUB' + return [message.data for ack_id, message in results.items()] class _FlattenEvaluator(_TransformEvaluator):