Revert "Throw specialized exception in value providers" This reverts commit ee92b9642bb6b6e42bb701ab638c55539163bb69.
Project: http://git-wip-us.apache.org/repos/asf/beam/repo Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/4854291f Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/4854291f Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/4854291f Branch: refs/heads/DSL_SQL Commit: 4854291fe38df7b624446311956d0ae0677f6b8c Parents: b7ddab2 Author: Ahmet Altay <[email protected]> Authored: Wed Apr 12 10:43:24 2017 -0700 Committer: Ahmet Altay <[email protected]> Committed: Wed Apr 12 11:50:39 2017 -0700 ---------------------------------------------------------------------- .../apache_beam/runners/dataflow/dataflow_runner.py | 6 ------ sdks/python/apache_beam/utils/value_provider.py | 14 ++------------ 2 files changed, 2 insertions(+), 18 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/beam/blob/4854291f/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py ---------------------------------------------------------------------- diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py index 4e81788..1a935c1 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py @@ -43,7 +43,6 @@ from apache_beam.runners.runner import PipelineState from apache_beam.transforms.display import DisplayData from apache_beam.typehints import typehints from apache_beam.utils.pipeline_options import StandardOptions -from apache_beam.utils.value_provider import RuntimeValueProviderError class DataflowRunner(PipelineRunner): @@ -477,11 +476,6 @@ class DataflowRunner(PipelineRunner): 'estimated_size_bytes': json_value.get_typed_value_descriptor( transform.source.estimate_size()) } - except RuntimeValueProviderError: - # Size estimation is best effort, and this error is by value provider. - logging.info( - 'Could not estimate size of source %r due to ' + \ - 'RuntimeValueProviderError', transform.source) except Exception: # pylint: disable=broad-except # Size estimation is best effort. So we log the error and continue. logging.info( http://git-wip-us.apache.org/repos/asf/beam/blob/4854291f/sdks/python/apache_beam/utils/value_provider.py ---------------------------------------------------------------------- diff --git a/sdks/python/apache_beam/utils/value_provider.py b/sdks/python/apache_beam/utils/value_provider.py index 271202d..a72fc4c 100644 --- a/sdks/python/apache_beam/utils/value_provider.py +++ b/sdks/python/apache_beam/utils/value_provider.py @@ -22,15 +22,6 @@ and dynamically provided values. from functools import wraps -class RuntimeValueProviderError(RuntimeError): - def __init__(self, msg): - """Class representing the errors thrown during runtime by the valueprovider - Args: - msg: Message string for the exception thrown - """ - super(RuntimeValueProviderError, self).__init__(msg) - - class ValueProvider(object): def is_accessible(self): raise NotImplementedError( @@ -76,8 +67,7 @@ class RuntimeValueProvider(ValueProvider): runtime_options = ( RuntimeValueProvider.runtime_options_map.get(self.options_id)) if runtime_options is None: - raise RuntimeValueProviderError( - '%s.get() not called from a runtime context' % self) + raise RuntimeError('%s.get() not called from a runtime context' % self) candidate = runtime_options.get(self.option_name) if candidate: @@ -114,7 +104,7 @@ def check_accessible(value_provider_list): def _f(self, *args, **kwargs): for obj in [getattr(self, vp) for vp in value_provider_list]: if not obj.is_accessible(): - raise RuntimeValueProviderError('%s not accessible' % obj) + raise RuntimeError('%s not accessible' % obj) return fnc(self, *args, **kwargs) return _f return _check_accessible
