From bc6b5252d34cab1633c34d23d38571ea9590dd20 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Wed, 22 Jun 2022 16:26:02 -0700 Subject: [PATCH 001/173] WIP: Created a skeleton dask runner implementation. --- .../apache_beam/options/pipeline_options.py | 1 + .../apache_beam/runners/dask/dask_runner.py | 123 ++++++++++++++++++ 2 files changed, 124 insertions(+) create mode 100644 sdks/python/apache_beam/runners/dask/dask_runner.py diff --git a/sdks/python/apache_beam/options/pipeline_options.py b/sdks/python/apache_beam/options/pipeline_options.py index ee74a94b2dd0..8622425625c5 100644 --- a/sdks/python/apache_beam/options/pipeline_options.py +++ b/sdks/python/apache_beam/options/pipeline_options.py @@ -455,6 +455,7 @@ class StandardOptions(PipelineOptions): 'apache_beam.runners.portability.spark_runner.SparkRunner', 'apache_beam.runners.test.TestDirectRunner', 'apache_beam.runners.test.TestDataflowRunner', + 'apache_beam.runners.dask.dask_runner.DaskRunner', ) KNOWN_RUNNER_NAMES = [path.split('.')[-1] for path in ALL_KNOWN_RUNNERS] diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py new file mode 100644 index 000000000000..b97426184e8c --- /dev/null +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -0,0 +1,123 @@ +# +# 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. +# + +"""DaskRunner, executing remote jobs on Dask.distributed. + +The DaskRunner is a runner implementation that executes a graph of +transformations across processes and workers via Dask distributed's +scheduler. +""" +import typing as t +import argparse +import dataclasses + +from apache_beam import pvalue +from apache_beam.options.pipeline_options import PipelineOptions +from apache_beam.pvalue import PValue +from apache_beam.runners.direct.consumer_tracking_pipeline_visitor import ConsumerTrackingPipelineVisitor +from apache_beam.utils.interactive_utils import is_in_notebook + +from apache_beam.runners.direct.direct_runner import BundleBasedDirectRunner + +from apache_beam.pipeline import PipelineVisitor, AppliedPTransform + +import dask.bag as db + + +class DaskOptions(PipelineOptions): + + @classmethod + def _add_argparse_args(cls, parser: argparse.ArgumentParser) -> None: + # TODO: get Dask client options + pass + + +@dataclasses.dataclass +class DaskExecutor: + value_to_consumers: t.Dict[pvalue.PValue, t.Set[AppliedPTransform]] + # root_transforms: t.Set[AppliedPTransform] + step_names: t.Dict[AppliedPTransform, str] + views: t.List[pvalue.AsSideInput] + + def __post_init__(self): + # TODO(alxr): Translate to Bags + + pass + + def start(self, roots: t.Set[AppliedPTransform]) -> None: + pass + + def await_completion(self) -> None: + pass + + def shutdown(self) -> None: + pass + + +class DaskRunner(BundleBasedDirectRunner): + """Executes a pipeline on a Dask distributed client.""" + + @staticmethod + def to_dask_bag_visitor(self): + + @dataclasses.dataclass + class DaskBagVisitor(PipelineVisitor): + + def visit_transform(self, transform_node: AppliedPTransform) -> None: + inputs = list(transform_node.inputs) + pass + + + + @staticmethod + def is_fnapi_compatible(): + return False + + def run_pipeline(self, pipeline, options): + # TODO(alxr): Create interactive notebook support. + if is_in_notebook(): + raise NotImplementedError('interactive support will come later!') + + try: + import dask.bag as db + import dask.distributed as ddist + except ImportError: + raise ImportError('DaskRunner is not available. Please install apache_beam[dask].') + + dask_options = options.view_as(DaskOptions) + + self.client = ddist.Client(**dask_options.get_all_options()) + + self.consumer_tracking_visitor = ConsumerTrackingPipelineVisitor() + pipeline.visit(self.consumer_tracking_visitor) + + dask_visitor = self.to_dask_bag_visitor() + pipeline.visit(dask_visitor) + + + # if pipeline: + # pass + # else: + # raise ValueError('Proto or FunctionAPI environments are not supported.') + # if pipeline: + # + # # Flatten / Optimize graph? + # + # # Trigger a traversal of all reachable nodes. + # self.visit_transforms(pipeline, options) + # + # Get API Client? \ No newline at end of file From 665ee616e73ece1af4fe6a147e5046621d7a46a8 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Thu, 23 Jun 2022 11:08:49 -0700 Subject: [PATCH 002/173] WIP: Idea for a translation evaluator. --- .../apache_beam/runners/dask/dask_runner.py | 56 +++++++++++++++---- .../runners/dask/transform_evaluator.py | 52 +++++++++++++++++ 2 files changed, 98 insertions(+), 10 deletions(-) create mode 100644 sdks/python/apache_beam/runners/dask/transform_evaluator.py diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index b97426184e8c..579d874a0df2 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -21,6 +21,8 @@ transformations across processes and workers via Dask distributed's scheduler. """ +import functools +import itertools import typing as t import argparse import dataclasses @@ -49,17 +51,27 @@ def _add_argparse_args(cls, parser: argparse.ArgumentParser) -> None: @dataclasses.dataclass class DaskExecutor: value_to_consumers: t.Dict[pvalue.PValue, t.Set[AppliedPTransform]] - # root_transforms: t.Set[AppliedPTransform] + root_transforms: t.Set[AppliedPTransform] step_names: t.Dict[AppliedPTransform, str] views: t.List[pvalue.AsSideInput] + _root_nodes = None + _all_nodes = None - def __post_init__(self): - # TODO(alxr): Translate to Bags + @property + @functools.cached_property + def root_nodes(self): + return frozenset(self.root_transforms) + + @property + @functools.cached_property + def all_nodes(self): + return itertools.chain( + self.root_nodes, *itertools.chain(self.value_to_consumers.values()))) + + def start(self) -> None: + for root in self.root_nodes: - pass - def start(self, roots: t.Set[AppliedPTransform]) -> None: - pass def await_completion(self) -> None: pass @@ -72,14 +84,38 @@ class DaskRunner(BundleBasedDirectRunner): """Executes a pipeline on a Dask distributed client.""" @staticmethod - def to_dask_bag_visitor(self): + def to_dask_bag_visitor(self) -> PipelineVisitor: @dataclasses.dataclass class DaskBagVisitor(PipelineVisitor): - def visit_transform(self, transform_node: AppliedPTransform) -> None: - inputs = list(transform_node.inputs) - pass + value_to_consumers = { + } # type: Dict[pvalue.PValue, Set[AppliedPTransform]] + root_transforms = set() # type: Set[AppliedPTransform] + step_names = {} # type: Dict[AppliedPTransform, str] + + def __post_init__(self): + self._num_transforms = 0 + self._views = set() + + def visit_transform(self, applied_ptransform: AppliedPTransform) -> None: + inputs = list(applied_ptransform.inputs) + if inputs: + for input_value in inputs: + if isinstance(input_value, pvalue.PBegin): + self.root_transforms.add(applied_ptransform) + if input_value not in self.value_to_consumers: + self.value_to_consumers[input_value] = set() + self.value_to_consumers[input_value].add(applied_ptransform) + else: + self.root_transforms.add(applied_ptransform) + self.step_names[applied_ptransform] = 's%d' % (self._num_transforms) + self._num_transforms += 1 + + for side_input in applied_ptransform.side_inputs: + self._views.add(side_input) + + return DaskBagVisitor() diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py new file mode 100644 index 000000000000..cc67bce5befe --- /dev/null +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -0,0 +1,52 @@ +import typing as t +import abc +import dataclasses + +import apache_beam +from apache_beam.pipeline import AppliedPTransform + +import dask.bag as db + + +@dataclasses.dataclass +class DaskOp(abc.ABC): + application: AppliedPTransform + side_inputs: t.Sequence[t.Any] + + @abc.abstractmethod + def apply(self, element): + pass + + +class NoOp(DaskOp): + def apply(self, element): + return element + + +class Create(DaskOp): + def apply(self, element): + assert element is None, 'Create expects no input!' + + original_transform = t.cast(apache_beam.Create, self.application.transform) + items = original_transform.values + return db.from_sequence(items) + + +class Impulse(DaskOp): + def apply(self, element): + raise NotImplementedError() + + +class ParDo(DaskOp): + def apply(self, element): + assert element is not None, 'ParDo must receive input!' + assert isinstance(element, db.Bag) + assert self.application is not None + transform = self.application.transform + assert isinstance(transform, apache_beam.ParDo) + + +TRANSLATIONS = { + apache_beam.Create: Create + +} \ No newline at end of file From d1095c7d2e86a04b26e19ba6f78c74eac3c61ccd Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 2 Jul 2022 00:35:38 -0700 Subject: [PATCH 003/173] Added overrides and a visitor that translates operations. --- .../apache_beam/runners/dask/dask_runner.py | 110 +++++--------- .../runners/dask/dask_runner_test.py | 50 +++++++ .../apache_beam/runners/dask/overrides.py | 139 ++++++++++++++++++ .../runners/dask/transform_evaluator.py | 50 +++++-- 4 files changed, 264 insertions(+), 85 deletions(-) create mode 100644 sdks/python/apache_beam/runners/dask/dask_runner_test.py create mode 100644 sdks/python/apache_beam/runners/dask/overrides.py diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index 579d874a0df2..daee5e6b65b5 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -20,24 +20,32 @@ The DaskRunner is a runner implementation that executes a graph of transformations across processes and workers via Dask distributed's scheduler. + + +Ideas to explore / Notes: +- Write a PCollection subclass that wraps a Dask Bag. + - Would be the input + return of the translation operators. +- The Ray runner is more focused on Task scheduling; This should focus more on graph translation. + +- A bundle is a subset of elements in a PCollection. i.e. a small set of elements that are processed together. +- In Dask, it is probably the same as a partition. Thus, we probably don't need to worry about it; Dask should take + care of it. """ -import functools -import itertools -import typing as t import argparse import dataclasses +import typing as t + +from dask import bag as db from apache_beam import pvalue from apache_beam.options.pipeline_options import PipelineOptions -from apache_beam.pvalue import PValue -from apache_beam.runners.direct.consumer_tracking_pipeline_visitor import ConsumerTrackingPipelineVisitor -from apache_beam.utils.interactive_utils import is_in_notebook - -from apache_beam.runners.direct.direct_runner import BundleBasedDirectRunner - from apache_beam.pipeline import PipelineVisitor, AppliedPTransform - -import dask.bag as db +from apache_beam.pvalue import PCollection +from apache_beam.runners.dask.overrides import dask_overrides +from apache_beam.runners.dask.transform_evaluator import TRANSLATIONS, NoOp +# from apache_beam.runners.direct.consumer_tracking_pipeline_visitor import ConsumerTrackingPipelineVisitor +from apache_beam.runners.direct.direct_runner import BundleBasedDirectRunner +from apache_beam.utils.interactive_utils import is_in_notebook class DaskOptions(PipelineOptions): @@ -48,77 +56,34 @@ def _add_argparse_args(cls, parser: argparse.ArgumentParser) -> None: pass -@dataclasses.dataclass -class DaskExecutor: - value_to_consumers: t.Dict[pvalue.PValue, t.Set[AppliedPTransform]] - root_transforms: t.Set[AppliedPTransform] - step_names: t.Dict[AppliedPTransform, str] - views: t.List[pvalue.AsSideInput] - _root_nodes = None - _all_nodes = None - - @property - @functools.cached_property - def root_nodes(self): - return frozenset(self.root_transforms) - - @property - @functools.cached_property - def all_nodes(self): - return itertools.chain( - self.root_nodes, *itertools.chain(self.value_to_consumers.values()))) - - def start(self) -> None: - for root in self.root_nodes: - - - - def await_completion(self) -> None: - pass - - def shutdown(self) -> None: - pass - - class DaskRunner(BundleBasedDirectRunner): """Executes a pipeline on a Dask distributed client.""" @staticmethod - def to_dask_bag_visitor(self) -> PipelineVisitor: + def to_dask_bag_visitor() -> PipelineVisitor: @dataclasses.dataclass class DaskBagVisitor(PipelineVisitor): + bags: t.Dict[AppliedPTransform, db.Bag] = dataclasses.fields(default_factory=dict) - value_to_consumers = { - } # type: Dict[pvalue.PValue, Set[AppliedPTransform]] - root_transforms = set() # type: Set[AppliedPTransform] - step_names = {} # type: Dict[AppliedPTransform, str] + def visit_transform(self, transform_node: AppliedPTransform) -> None: + op_class = TRANSLATIONS.get(transform_node.transform.__class__, NoOp) + op = op_class(transform_node) - def __post_init__(self): - self._num_transforms = 0 - self._views = set() - - def visit_transform(self, applied_ptransform: AppliedPTransform) -> None: - inputs = list(applied_ptransform.inputs) + inputs = list(transform_node.inputs) if inputs: for input_value in inputs: if isinstance(input_value, pvalue.PBegin): - self.root_transforms.add(applied_ptransform) - if input_value not in self.value_to_consumers: - self.value_to_consumers[input_value] = set() - self.value_to_consumers[input_value].add(applied_ptransform) - else: - self.root_transforms.add(applied_ptransform) - self.step_names[applied_ptransform] = 's%d' % (self._num_transforms) - self._num_transforms += 1 + self.bags[transform_node] = op.apply(None) - for side_input in applied_ptransform.side_inputs: - self._views.add(side_input) + prev_op = input_value.producer + if prev_op in self.bags: + self.bags[transform_node] = op.apply(self.bags[prev_op]) + else: + self.bags[transform_node] = op.apply(None) return DaskBagVisitor() - - @staticmethod def is_fnapi_compatible(): return False @@ -134,16 +99,19 @@ def run_pipeline(self, pipeline, options): except ImportError: raise ImportError('DaskRunner is not available. Please install apache_beam[dask].') - dask_options = options.view_as(DaskOptions) + # TODO(alxr): Wire up a real dask client + # dask_options = options.view_as(DaskOptions).get_all_options() + # self.client = ddist.Client(**dask_options) - self.client = ddist.Client(**dask_options.get_all_options()) + pipeline.replace_all(dask_overrides()) - self.consumer_tracking_visitor = ConsumerTrackingPipelineVisitor() - pipeline.visit(self.consumer_tracking_visitor) + # consumer_tracking_visitor = ConsumerTrackingPipelineVisitor() + # pipeline.visit(consumer_tracking_visitor) dask_visitor = self.to_dask_bag_visitor() pipeline.visit(dask_visitor) + print(dask_visitor) # if pipeline: # pass @@ -156,4 +124,4 @@ def run_pipeline(self, pipeline, options): # # Trigger a traversal of all reachable nodes. # self.visit_transforms(pipeline, options) # - # Get API Client? \ No newline at end of file + # Get API Client? diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py new file mode 100644 index 000000000000..377e3b37bbaf --- /dev/null +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -0,0 +1,50 @@ +# +# 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.runners.dask.dask_runner import DaskRunner +from apache_beam.testing import test_pipeline + + +class DaskRunnerRunPipelineTest(unittest.TestCase): + """Test class used to introspect the dask runner via a debugger.""" + + def test_create(self): + p = test_pipeline.TestPipeline(runner=DaskRunner()) + _ = p | beam.Create([1]) + p.run() + + def test_create_and_map(self): + def double(x): + return x * 2 + + p = test_pipeline.TestPipeline(runner=DaskRunner()) + _ = p | beam.Create([1]) | beam.Map(double) + p.run() + + def test_create_map_and_groupby(self): + def double(x): + return x * 2, x + + p = test_pipeline.TestPipeline(runner=DaskRunner()) + _ = p | beam.Create([1]) | beam.Map(double) | beam.GroupByKey() + p.run() + + +if __name__ == '__main__': + unittest.main() diff --git a/sdks/python/apache_beam/runners/dask/overrides.py b/sdks/python/apache_beam/runners/dask/overrides.py new file mode 100644 index 000000000000..38f4701ed76a --- /dev/null +++ b/sdks/python/apache_beam/runners/dask/overrides.py @@ -0,0 +1,139 @@ +# +# 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 dataclasses +import typing as t + +import apache_beam as beam +from apache_beam import ( + Create, + GroupByKey, + ParDo, + PTransform, + pvalue, + Windowing, +) +from apache_beam import typehints +from apache_beam.io.iobase import SourceBase +from apache_beam.pipeline import PTransformOverride, AppliedPTransform +from apache_beam.runners.direct.direct_runner import _GroupAlsoByWindowDoFn +from apache_beam.transforms import ptransform +from apache_beam.transforms.window import GlobalWindows + +K = t.TypeVar("K") +V = t.TypeVar("V") + + +@dataclasses.dataclass +class _Create(PTransform): + values: t.Tuple[t.Any] + + def expand(self, input_or_inputs): + return pvalue.PCollection.from_(input_or_inputs) + + def get_windowing(self, inputs): + # type: (typing.Any) -> Windowing + return Windowing(GlobalWindows()) + + +@typehints.with_input_types(K) +@typehints.with_output_types(K) +class _Reshuffle(PTransform): + def expand(self, input_or_inputs): + return pvalue.PCollection.from_(input_or_inputs) + + +@dataclasses.dataclass +class _Read(PTransform): + source: SourceBase + + def expand(self, input_or_inputs): + return pvalue.PCollection.from_(input_or_inputs) + + +@typehints.with_input_types(t.Tuple[K, V]) +@typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) +class _GroupByKeyOnly(PTransform): + def expand(self, input_or_inputs): + return pvalue.PCollection.from_(input_or_inputs) + + def infer_output_type(self, input_type): + key_type, value_type = typehints.trivial_inference.key_value_types(input_type) + return typehints.KV[key_type, typehints.Iterable[value_type]] + + +@typehints.with_input_types(t.Tuple[K, t.Iterable[V]]) +@typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) +class _GroupAlsoByWindow(ParDo): + def __init__(self, windowing): + super(_GroupAlsoByWindow, self).__init__( + _GroupAlsoByWindowDoFn(windowing)) + self.windowing = windowing + + def expand(self, input_or_inputs): + return pvalue.PCollection.from_(input_or_inputs) + + +@typehints.with_input_types(t.Tuple[K, V]) +@typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) +class _GroupByKey(PTransform): + def expand(self, input_or_inputs): + return ( + input_or_inputs + | "ReifyWindows" >> ParDo(GroupByKey.ReifyWindows()) + | "GroupByKey" >> _GroupByKeyOnly() + | "GroupByWindow" >> _GroupAlsoByWindow(input_or_inputs.windowing)) + + +def dask_overrides() -> t.List[PTransformOverride]: + class CreateOverride(PTransformOverride): + def matches(self, applied_ptransform: AppliedPTransform) -> bool: + return applied_ptransform.transform.__class__ == beam.Create + + def get_replacement_transform_for_applied_ptransform( + self, applied_ptransform: AppliedPTransform) -> ptransform.PTransform: + return _Create(t.cast(Create, applied_ptransform.transform).values) + + class ReshuffleOverride(PTransformOverride): + def matches(self, applied_ptransform: AppliedPTransform) -> bool: + return applied_ptransform.transform.__class__ == beam.Reshuffle + + def get_replacement_transform_for_applied_ptransform( + self, applied_ptransform: AppliedPTransform) -> ptransform.PTransform: + return _Reshuffle() + + class ReadOverride(PTransformOverride): + def matches(self, applied_ptransform: AppliedPTransform) -> bool: + return applied_ptransform.transform.__class__ == beam.io.Read + + def get_replacement_transform_for_applied_ptransform( + self, applied_ptransform: AppliedPTransform) -> ptransform.PTransform: + return _Read(t.cast(beam.io.Read, applied_ptransform.transform).source) + + class GroupByKeyOverride(PTransformOverride): + def matches(self, applied_ptransform: AppliedPTransform) -> bool: + return applied_ptransform.transform.__class__ == beam.GroupByKey + + def get_replacement_transform_for_applied_ptransform( + self, applied_ptransform: AppliedPTransform) -> ptransform.PTransform: + return _GroupByKey() + + return [ + CreateOverride(), + ReshuffleOverride(), + ReadOverride(), + GroupByKeyOverride(), + ] diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index cc67bce5befe..fbcf498eb051 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -1,3 +1,19 @@ +# +# 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 typing as t import abc import dataclasses @@ -7,24 +23,29 @@ import dask.bag as db +from apache_beam.runners.dask.overrides import _Create + @dataclasses.dataclass -class DaskOp(abc.ABC): +class DaskBagOp(abc.ABC): application: AppliedPTransform - side_inputs: t.Sequence[t.Any] + + @property + def side_inputs(self): + return self.application.side_inputs @abc.abstractmethod - def apply(self, element): + def apply(self, element: db.Bag) -> db.Bag: pass -class NoOp(DaskOp): - def apply(self, element): +class NoOp(DaskBagOp): + def apply(self, element: db.Bag) -> db.Bag: return element -class Create(DaskOp): - def apply(self, element): +class Create(DaskBagOp): + def apply(self, element: db.Bag) -> db.Bag: assert element is None, 'Create expects no input!' original_transform = t.cast(apache_beam.Create, self.application.transform) @@ -32,21 +53,22 @@ def apply(self, element): return db.from_sequence(items) -class Impulse(DaskOp): - def apply(self, element): +class Impulse(DaskBagOp): + def apply(self, element: db.Bag) -> db.Bag: raise NotImplementedError() -class ParDo(DaskOp): - def apply(self, element): +class ParDo(DaskBagOp): + def apply(self, element: db.Bag) -> db.Bag: assert element is not None, 'ParDo must receive input!' assert isinstance(element, db.Bag) assert self.application is not None transform = self.application.transform assert isinstance(transform, apache_beam.ParDo) + return element -TRANSLATIONS = { - apache_beam.Create: Create -} \ No newline at end of file +TRANSLATIONS = { + _Create: Create +} From fd13d40ec4604d36989ddf6f14686e5728f099b2 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 2 Jul 2022 00:36:43 -0700 Subject: [PATCH 004/173] Fixed a dataclass typo. --- sdks/python/apache_beam/runners/dask/dask_runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index daee5e6b65b5..0693a98bbec3 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -64,7 +64,7 @@ def to_dask_bag_visitor() -> PipelineVisitor: @dataclasses.dataclass class DaskBagVisitor(PipelineVisitor): - bags: t.Dict[AppliedPTransform, db.Bag] = dataclasses.fields(default_factory=dict) + bags: t.Dict[AppliedPTransform, db.Bag] = dataclasses.field(default_factory=dict) def visit_transform(self, transform_node: AppliedPTransform) -> None: op_class = TRANSLATIONS.get(transform_node.transform.__class__, NoOp) From e3d0c8a3b6929c4f2a76927f5098d667d6a1f265 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 2 Jul 2022 01:06:54 -0700 Subject: [PATCH 005/173] Expanded translations. --- .../runners/dask/transform_evaluator.py | 54 +++++++++++-------- 1 file changed, 32 insertions(+), 22 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index fbcf498eb051..3f500e0d8e18 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -14,6 +14,10 @@ # See the License for the specific language governing permissions and # limitations under the License. # +""" + +TODO(alxr): Translate ops from https://docs.dask.org/en/latest/bag-api.html. +""" import typing as t import abc import dataclasses @@ -23,52 +27,58 @@ import dask.bag as db -from apache_beam.runners.dask.overrides import _Create +from apache_beam.runners.dask.overrides import _Create, _GroupByKeyOnly @dataclasses.dataclass class DaskBagOp(abc.ABC): - application: AppliedPTransform + applied: AppliedPTransform @property def side_inputs(self): - return self.application.side_inputs + return self.applied.side_inputs @abc.abstractmethod - def apply(self, element: db.Bag) -> db.Bag: + def apply(self, input_bag: t.Optional[db.Bag]) -> db.Bag: pass class NoOp(DaskBagOp): - def apply(self, element: db.Bag) -> db.Bag: - return element + def apply(self, input_bag: t.Optional[db.Bag]) -> db.Bag: + return input_bag class Create(DaskBagOp): - def apply(self, element: db.Bag) -> db.Bag: - assert element is None, 'Create expects no input!' - - original_transform = t.cast(apache_beam.Create, self.application.transform) + def apply(self, input_bag: t.Optional[db.Bag]) -> db.Bag: + assert input_bag is None, 'Create expects no input!' + original_transform = t.cast(_Create, self.applied.transform) items = original_transform.values return db.from_sequence(items) -class Impulse(DaskBagOp): - def apply(self, element: db.Bag) -> db.Bag: - raise NotImplementedError() +class ParDo(DaskBagOp): + def apply(self, input_bag: t.Optional[db.Bag]) -> db.Bag: + fn = t.cast(apache_beam.ParDo, self.applied.transform).fn + return input_bag.map(fn).flatten() + +class Map(DaskBagOp): + def apply(self, input_bag: t.Optional[db.Bag]) -> db.Bag: + fn = t.cast(apache_beam.Map, self.applied.transform).fn + return input_bag.map(fn) -class ParDo(DaskBagOp): - def apply(self, element: db.Bag) -> db.Bag: - assert element is not None, 'ParDo must receive input!' - assert isinstance(element, db.Bag) - assert self.application is not None - transform = self.application.transform - assert isinstance(transform, apache_beam.ParDo) - return element +class GroupByKey(DaskBagOp): + def apply(self, input_bag: t.Optional[db.Bag]) -> db.Bag: + def key(item): + return item[0] + + return input_bag.groupby(key) TRANSLATIONS = { - _Create: Create + _Create: Create, + apache_beam.ParDo: ParDo, + apache_beam.Map: Map, + _GroupByKeyOnly: GroupByKey, } From 69a660f03c0016072b863b89600574a8f4a60adc Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 2 Jul 2022 01:33:30 -0700 Subject: [PATCH 006/173] Core idea seems to be kinda working... --- .../apache_beam/runners/dask/__init__.py | 0 .../apache_beam/runners/dask/dask_runner.py | 22 +++++++++---------- .../apache_beam/runners/dask/overrides.py | 4 +++- .../runners/dask/transform_evaluator.py | 4 ++-- 4 files changed, 15 insertions(+), 15 deletions(-) create mode 100644 sdks/python/apache_beam/runners/dask/__init__.py diff --git a/sdks/python/apache_beam/runners/dask/__init__.py b/sdks/python/apache_beam/runners/dask/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index 0693a98bbec3..1b4910a25af9 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -35,15 +35,11 @@ import dataclasses import typing as t -from dask import bag as db - from apache_beam import pvalue from apache_beam.options.pipeline_options import PipelineOptions from apache_beam.pipeline import PipelineVisitor, AppliedPTransform -from apache_beam.pvalue import PCollection from apache_beam.runners.dask.overrides import dask_overrides from apache_beam.runners.dask.transform_evaluator import TRANSLATIONS, NoOp -# from apache_beam.runners.direct.consumer_tracking_pipeline_visitor import ConsumerTrackingPipelineVisitor from apache_beam.runners.direct.direct_runner import BundleBasedDirectRunner from apache_beam.utils.interactive_utils import is_in_notebook @@ -60,7 +56,9 @@ class DaskRunner(BundleBasedDirectRunner): """Executes a pipeline on a Dask distributed client.""" @staticmethod - def to_dask_bag_visitor() -> PipelineVisitor: + def to_dask_bag_visitor() -> 'DaskBagVisitor': + + from dask import bag as db @dataclasses.dataclass class DaskBagVisitor(PipelineVisitor): @@ -99,20 +97,20 @@ def run_pipeline(self, pipeline, options): except ImportError: raise ImportError('DaskRunner is not available. Please install apache_beam[dask].') - # TODO(alxr): Wire up a real dask client - # dask_options = options.view_as(DaskOptions).get_all_options() - # self.client = ddist.Client(**dask_options) + # TODO(alxr): Actually use this right. + dask_options = options.view_as(DaskOptions).get_all_options(drop_default=True) + client = ddist.Client(**dask_options) pipeline.replace_all(dask_overrides()) - # consumer_tracking_visitor = ConsumerTrackingPipelineVisitor() - # pipeline.visit(consumer_tracking_visitor) - dask_visitor = self.to_dask_bag_visitor() pipeline.visit(dask_visitor) - print(dask_visitor) + for bag in dask_visitor.bags.values(): + bag.compute() + # TODO(alxr): Return the proper thing... + return None # if pipeline: # pass # else: diff --git a/sdks/python/apache_beam/runners/dask/overrides.py b/sdks/python/apache_beam/runners/dask/overrides.py index 38f4701ed76a..30df036eb7d9 100644 --- a/sdks/python/apache_beam/runners/dask/overrides.py +++ b/sdks/python/apache_beam/runners/dask/overrides.py @@ -78,6 +78,7 @@ def infer_output_type(self, input_type): @typehints.with_input_types(t.Tuple[K, t.Iterable[V]]) @typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) class _GroupAlsoByWindow(ParDo): + """Not used yet...""" def __init__(self, windowing): super(_GroupAlsoByWindow, self).__init__( _GroupAlsoByWindowDoFn(windowing)) @@ -90,6 +91,7 @@ def expand(self, input_or_inputs): @typehints.with_input_types(t.Tuple[K, V]) @typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) class _GroupByKey(PTransform): + """Not used yet...""" def expand(self, input_or_inputs): return ( input_or_inputs @@ -129,7 +131,7 @@ def matches(self, applied_ptransform: AppliedPTransform) -> bool: def get_replacement_transform_for_applied_ptransform( self, applied_ptransform: AppliedPTransform) -> ptransform.PTransform: - return _GroupByKey() + return _GroupByKeyOnly() return [ CreateOverride(), diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 3f500e0d8e18..e5b6ae00aba2 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -59,13 +59,13 @@ def apply(self, input_bag: t.Optional[db.Bag]) -> db.Bag: class ParDo(DaskBagOp): def apply(self, input_bag: t.Optional[db.Bag]) -> db.Bag: fn = t.cast(apache_beam.ParDo, self.applied.transform).fn - return input_bag.map(fn).flatten() + return input_bag.map(fn.process).flatten() class Map(DaskBagOp): def apply(self, input_bag: t.Optional[db.Bag]) -> db.Bag: fn = t.cast(apache_beam.Map, self.applied.transform).fn - return input_bag.map(fn) + return input_bag.map(fn.process) class GroupByKey(DaskBagOp): From ab583348d4fe80fc266e8ad9f190465155f3862f Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sun, 3 Jul 2022 15:02:10 -0700 Subject: [PATCH 007/173] First iteration on DaskRunnerResult (keep track of pipeline state). --- .../apache_beam/runners/dask/dask_runner.py | 39 ++++++++++++++++--- .../runners/dask/dask_runner_test.py | 18 ++++----- 2 files changed, 42 insertions(+), 15 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index 1b4910a25af9..d70f4c3f5638 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -41,6 +41,7 @@ from apache_beam.runners.dask.overrides import dask_overrides from apache_beam.runners.dask.transform_evaluator import TRANSLATIONS, NoOp from apache_beam.runners.direct.direct_runner import BundleBasedDirectRunner +from apache_beam.runners.runner import PipelineResult, PipelineState from apache_beam.utils.interactive_utils import is_in_notebook @@ -48,16 +49,43 @@ class DaskOptions(PipelineOptions): @classmethod def _add_argparse_args(cls, parser: argparse.ArgumentParser) -> None: - # TODO: get Dask client options + # TODO(alxr): get Dask client options pass +@dataclasses.dataclass +class DaskRunnerResult(PipelineResult): + client: 'dask.distributed.Client' + futures: t.Sequence['dask.distributed.Future'] + + def __post_init__(self): + super().__init__(PipelineState.RUNNING) + + def wait_until_finish(self, duration=None) -> PipelineState: + try: + self.client.wait_for_workers(timeout=(duration / 1000)) + self._state = PipelineState.DONE + except: # pylint: disable=broad-except + self._state = PipelineState.FAILED + raise + return self._state + + def cancel(self) -> PipelineState: + self._state = PipelineState.CANCELLING + self.client.cancel(self.futures) + self._state = PipelineState.CANCELLED + return self._state + + def metrics(self): + # TODO(alxr): Collect and return metrics... + raise NotImplementedError('collecting metrics will come later!') + + class DaskRunner(BundleBasedDirectRunner): """Executes a pipeline on a Dask distributed client.""" @staticmethod def to_dask_bag_visitor() -> 'DaskBagVisitor': - from dask import bag as db @dataclasses.dataclass @@ -106,11 +134,10 @@ def run_pipeline(self, pipeline, options): dask_visitor = self.to_dask_bag_visitor() pipeline.visit(dask_visitor) - for bag in dask_visitor.bags.values(): - bag.compute() + futures = client.compute(list(dask_visitor.bags.values())) + + return DaskRunnerResult(client, futures) - # TODO(alxr): Return the proper thing... - return None # if pipeline: # pass # else: diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index 377e3b37bbaf..eab347bf5e7f 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -24,26 +24,26 @@ class DaskRunnerRunPipelineTest(unittest.TestCase): """Test class used to introspect the dask runner via a debugger.""" + def setUp(self) -> None: + self.p = test_pipeline.TestPipeline(runner=DaskRunner()) + def test_create(self): - p = test_pipeline.TestPipeline(runner=DaskRunner()) - _ = p | beam.Create([1]) - p.run() + _ = self.p | beam.Create([1]) + self.p.run() def test_create_and_map(self): def double(x): return x * 2 - p = test_pipeline.TestPipeline(runner=DaskRunner()) - _ = p | beam.Create([1]) | beam.Map(double) - p.run() + _ = self.p | beam.Create([1]) | beam.Map(double) + self.p.run() def test_create_map_and_groupby(self): def double(x): return x * 2, x - p = test_pipeline.TestPipeline(runner=DaskRunner()) - _ = p | beam.Create([1]) | beam.Map(double) | beam.GroupByKey() - p.run() + _ = self.p | beam.Create([1]) | beam.Map(double) | beam.GroupByKey() + self.p.run() if __name__ == '__main__': From 5391cd607bcd160e6346a66cfbae5c48db2e5079 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sun, 3 Jul 2022 21:16:24 -0700 Subject: [PATCH 008/173] Added minimal set of DaskRunner options. --- .../apache_beam/runners/dask/dask_runner.py | 30 ++++++++++++++++--- 1 file changed, 26 insertions(+), 4 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index d70f4c3f5638..cc8148e9edd4 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -14,7 +14,6 @@ # See the License for the specific language governing permissions and # limitations under the License. # - """DaskRunner, executing remote jobs on Dask.distributed. The DaskRunner is a runner implementation that executes a graph of @@ -35,6 +34,8 @@ import dataclasses import typing as t +import dask.distributed + from apache_beam import pvalue from apache_beam.options.pipeline_options import PipelineOptions from apache_beam.pipeline import PipelineVisitor, AppliedPTransform @@ -47,10 +48,28 @@ class DaskOptions(PipelineOptions): + @staticmethod + def _parse_timeout(candidate): + try: + return int(candidate) + except: # noqa + import dask + return dask.config.no_default + @classmethod def _add_argparse_args(cls, parser: argparse.ArgumentParser) -> None: - # TODO(alxr): get Dask client options - pass + parser.add_argument('--dask_client_address', type=str, default=None, + help='Address of a dask Scheduler server. Will default to a `dask.LocalCluster()`.') + parser.add_argument('--dask_connection_timeout', dest='timeout', type=DaskOptions._parse_timeout, + help='Timeout duration for initial connection to the scheduler.') + parser.add_argument('--dask_scheduler_file', type=str, default=None, + help='Path to a file with scheduler information if available.') + # TODO(alxr): Add options for security. + parser.add_argument('--dask_client_name', dest='name', type=str, default=None, + help='Gives the client a name that will be included in logs generated on the scheduler ' + 'for matters relating to this client.') + parser.add_argument('--dask_connection_limit', dest='connection_limit', type=int, default=512, + help='The number of open comms to maintain at once in the connection pool.') @dataclasses.dataclass @@ -63,7 +82,10 @@ def __post_init__(self): def wait_until_finish(self, duration=None) -> PipelineState: try: - self.client.wait_for_workers(timeout=(duration / 1000)) + if duration is not None: + # Convert milliseconds to seconds + duration /= 1000 + self.client.wait_for_workers(timeout=duration) self._state = PipelineState.DONE except: # pylint: disable=broad-except self._state = PipelineState.FAILED From 5deb5986a9c8ab5347fefdcfc010ce928f9ae0da Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Fri, 8 Jul 2022 15:15:08 -0700 Subject: [PATCH 009/173] WIP: Alllmost got asserts to work! The current status is: - CoGroupByKey is broken due to how tags are used with GroupByKey - GroupByKey should output `[('0', None), ('1', 1)]`, however it actually outputs: [(None, ('1', 1)), (None, ('0', None))] - Once that is fixed, we may have test pipelines work on Dask. --- .../apache_beam/runners/dask/dask_runner.py | 12 ++++++-- .../runners/dask/dask_runner_test.py | 7 +++-- .../apache_beam/runners/dask/overrides.py | 30 +++++++++++++++---- .../runners/dask/transform_evaluator.py | 20 +++++++++---- 4 files changed, 55 insertions(+), 14 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index cc8148e9edd4..4caab2cd475f 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -86,6 +86,7 @@ def wait_until_finish(self, duration=None) -> PipelineState: # Convert milliseconds to seconds duration /= 1000 self.client.wait_for_workers(timeout=duration) + self.client.gather(self.futures, errors='raise') self._state = PipelineState.DONE except: # pylint: disable=broad-except self._state = PipelineState.FAILED @@ -120,13 +121,20 @@ def visit_transform(self, transform_node: AppliedPTransform) -> None: inputs = list(transform_node.inputs) if inputs: + bag_inputs = [] for input_value in inputs: if isinstance(input_value, pvalue.PBegin): - self.bags[transform_node] = op.apply(None) + bag_inputs.append(None) prev_op = input_value.producer if prev_op in self.bags: - self.bags[transform_node] = op.apply(self.bags[prev_op]) + bag_inputs.append(self.bags[prev_op]) + + if len(bag_inputs) == 1: + self.bags[transform_node] = op.apply(bag_inputs[0]) + else: + self.bags[transform_node] = op.apply(bag_inputs) + else: self.bags[transform_node] = op.apply(None) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index eab347bf5e7f..a3788348336d 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -17,8 +17,10 @@ import unittest import apache_beam as beam +from apache_beam.runners import DirectRunner from apache_beam.runners.dask.dask_runner import DaskRunner from apache_beam.testing import test_pipeline +from apache_beam.testing.util import assert_that, equal_to class DaskRunnerRunPipelineTest(unittest.TestCase): @@ -28,8 +30,9 @@ def setUp(self) -> None: self.p = test_pipeline.TestPipeline(runner=DaskRunner()) def test_create(self): - _ = self.p | beam.Create([1]) - self.p.run() + with self.p as p: + pcoll = p | beam.Create([1]) + assert_that(pcoll, equal_to([1])) def test_create_and_map(self): def double(x): diff --git a/sdks/python/apache_beam/runners/dask/overrides.py b/sdks/python/apache_beam/runners/dask/overrides.py index 30df036eb7d9..ed88184a98f7 100644 --- a/sdks/python/apache_beam/runners/dask/overrides.py +++ b/sdks/python/apache_beam/runners/dask/overrides.py @@ -15,6 +15,7 @@ # limitations under the License. # import dataclasses +import time import typing as t import apache_beam as beam @@ -32,6 +33,8 @@ from apache_beam.runners.direct.direct_runner import _GroupAlsoByWindowDoFn from apache_beam.transforms import ptransform from apache_beam.transforms.window import GlobalWindows +from apache_beam.typehints import TypeCheckError +from apache_beam.utils.windowed_value import WindowedValue K = t.TypeVar("K") V = t.TypeVar("V") @@ -78,7 +81,7 @@ def infer_output_type(self, input_type): @typehints.with_input_types(t.Tuple[K, t.Iterable[V]]) @typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) class _GroupAlsoByWindow(ParDo): - """Not used yet...""" + def __init__(self, windowing): super(_GroupAlsoByWindow, self).__init__( _GroupAlsoByWindowDoFn(windowing)) @@ -91,13 +94,21 @@ def expand(self, input_or_inputs): @typehints.with_input_types(t.Tuple[K, V]) @typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) class _GroupByKey(PTransform): - """Not used yet...""" + def expand(self, input_or_inputs): return ( input_or_inputs - | "ReifyWindows" >> ParDo(GroupByKey.ReifyWindows()) + # | "ReifyWindows" >> ParDo(GroupByKey.ReifyWindows()) | "GroupByKey" >> _GroupByKeyOnly() - | "GroupByWindow" >> _GroupAlsoByWindow(input_or_inputs.windowing)) + # | "GetValue" >> beam.Map(lambda p: p[1]) + # | "GroupByWindow" >> _GroupAlsoByWindow(input_or_inputs.windowing) + ) + + +class _Flatten(PTransform): + def expand(self, input_or_inputs): + is_bounded = all(pcoll.is_bounded for pcoll in input_or_inputs) + return pvalue.PCollection(self.pipeline, is_bounded=is_bounded) def dask_overrides() -> t.List[PTransformOverride]: @@ -131,11 +142,20 @@ def matches(self, applied_ptransform: AppliedPTransform) -> bool: def get_replacement_transform_for_applied_ptransform( self, applied_ptransform: AppliedPTransform) -> ptransform.PTransform: - return _GroupByKeyOnly() + return _GroupByKey() + + class FlattenOverride(PTransformOverride): + def matches(self, applied_ptransform: AppliedPTransform) -> bool: + return applied_ptransform.transform.__class__ == beam.Flatten + + def get_replacement_transform_for_applied_ptransform( + self, applied_ptransform: AppliedPTransform) -> ptransform.PTransform: + return _Flatten() return [ CreateOverride(), ReshuffleOverride(), ReadOverride(), GroupByKeyOverride(), + FlattenOverride(), ] diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index e5b6ae00aba2..762db2d81044 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -18,16 +18,19 @@ TODO(alxr): Translate ops from https://docs.dask.org/en/latest/bag-api.html. """ -import typing as t import abc import dataclasses - -import apache_beam -from apache_beam.pipeline import AppliedPTransform +import typing as t import dask.bag as db -from apache_beam.runners.dask.overrides import _Create, _GroupByKeyOnly +import apache_beam +from apache_beam.pipeline import AppliedPTransform +from apache_beam.runners.dask.overrides import ( + _Create, + _GroupByKeyOnly, + _Flatten +) @dataclasses.dataclass @@ -76,9 +79,16 @@ def key(item): return input_bag.groupby(key) +class Flatten(DaskBagOp): + def apply(self, input_bag: t.Optional[t.Sequence[db.Bag]]) -> db.Bag: + assert type(input_bag) is list, 'Must take a sequence of bags!' + return db.concat(input_bag) + + TRANSLATIONS = { _Create: Create, apache_beam.ParDo: ParDo, apache_beam.Map: Map, _GroupByKeyOnly: GroupByKey, + _Flatten: Flatten, } From 1768e4753976e6e132987b41fae8ca76a8c0f421 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Fri, 8 Jul 2022 15:34:03 -0700 Subject: [PATCH 010/173] With a great 1-liner from @pabloem, groupby is fixed! Now, all three initial tests pass. --- .../apache_beam/runners/dask/dask_runner_test.py | 16 ++++++++++------ .../runners/dask/transform_evaluator.py | 6 +++++- 2 files changed, 15 insertions(+), 7 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index a3788348336d..fcfc59f59bfa 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -27,10 +27,10 @@ class DaskRunnerRunPipelineTest(unittest.TestCase): """Test class used to introspect the dask runner via a debugger.""" def setUp(self) -> None: - self.p = test_pipeline.TestPipeline(runner=DaskRunner()) + self.pipeline = test_pipeline.TestPipeline(runner=DaskRunner()) def test_create(self): - with self.p as p: + with self.pipeline as p: pcoll = p | beam.Create([1]) assert_that(pcoll, equal_to([1])) @@ -38,15 +38,19 @@ def test_create_and_map(self): def double(x): return x * 2 - _ = self.p | beam.Create([1]) | beam.Map(double) - self.p.run() + with self.pipeline as p: + pcoll = p | beam.Create([1]) | beam.Map(double) + assert_that(pcoll, equal_to([2])) def test_create_map_and_groupby(self): def double(x): return x * 2, x - _ = self.p | beam.Create([1]) | beam.Map(double) | beam.GroupByKey() - self.p.run() + with self.pipeline as p: + pcoll = p | beam.Create([1]) | beam.Map(double) | beam.GroupByKey() + assert_that(pcoll, equal_to([ + (2, [1]) + ])) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 762db2d81044..4f1f02d22e94 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -76,7 +76,11 @@ def apply(self, input_bag: t.Optional[db.Bag]) -> db.Bag: def key(item): return item[0] - return input_bag.groupby(key) + def value(item): + k, v = item + return k, [elm[1] for elm in v] + + return input_bag.groupby(key).map(value) class Flatten(DaskBagOp): From e6c71062ea0df2f37e454dbdf528451372af7d9f Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Fri, 8 Jul 2022 15:40:15 -0700 Subject: [PATCH 011/173] Self-review: Cleaned up dask runner impl. --- .../apache_beam/options/pipeline_options.py | 1 - sdks/python/apache_beam/runners/dask/__init__.py | 16 ++++++++++++++++ .../apache_beam/runners/dask/dask_runner.py | 10 ---------- .../apache_beam/runners/dask/dask_runner_test.py | 1 - .../python/apache_beam/runners/dask/overrides.py | 8 +------- .../runners/dask/transform_evaluator.py | 16 +++++++++------- 6 files changed, 26 insertions(+), 26 deletions(-) diff --git a/sdks/python/apache_beam/options/pipeline_options.py b/sdks/python/apache_beam/options/pipeline_options.py index 8622425625c5..ee74a94b2dd0 100644 --- a/sdks/python/apache_beam/options/pipeline_options.py +++ b/sdks/python/apache_beam/options/pipeline_options.py @@ -455,7 +455,6 @@ class StandardOptions(PipelineOptions): 'apache_beam.runners.portability.spark_runner.SparkRunner', 'apache_beam.runners.test.TestDirectRunner', 'apache_beam.runners.test.TestDataflowRunner', - 'apache_beam.runners.dask.dask_runner.DaskRunner', ) KNOWN_RUNNER_NAMES = [path.split('.')[-1] for path in ALL_KNOWN_RUNNERS] diff --git a/sdks/python/apache_beam/runners/dask/__init__.py b/sdks/python/apache_beam/runners/dask/__init__.py index e69de29bb2d1..cce3acad34a4 100644 --- a/sdks/python/apache_beam/runners/dask/__init__.py +++ b/sdks/python/apache_beam/runners/dask/__init__.py @@ -0,0 +1,16 @@ +# +# 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. +# diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index 4caab2cd475f..e77d223f178c 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -19,16 +19,6 @@ The DaskRunner is a runner implementation that executes a graph of transformations across processes and workers via Dask distributed's scheduler. - - -Ideas to explore / Notes: -- Write a PCollection subclass that wraps a Dask Bag. - - Would be the input + return of the translation operators. -- The Ray runner is more focused on Task scheduling; This should focus more on graph translation. - -- A bundle is a subset of elements in a PCollection. i.e. a small set of elements that are processed together. -- In Dask, it is probably the same as a partition. Thus, we probably don't need to worry about it; Dask should take - care of it. """ import argparse import dataclasses diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index fcfc59f59bfa..c731a8dae51d 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -17,7 +17,6 @@ import unittest import apache_beam as beam -from apache_beam.runners import DirectRunner from apache_beam.runners.dask.dask_runner import DaskRunner from apache_beam.testing import test_pipeline from apache_beam.testing.util import assert_that, equal_to diff --git a/sdks/python/apache_beam/runners/dask/overrides.py b/sdks/python/apache_beam/runners/dask/overrides.py index ed88184a98f7..3b172535bfe2 100644 --- a/sdks/python/apache_beam/runners/dask/overrides.py +++ b/sdks/python/apache_beam/runners/dask/overrides.py @@ -15,13 +15,11 @@ # limitations under the License. # import dataclasses -import time import typing as t import apache_beam as beam from apache_beam import ( Create, - GroupByKey, ParDo, PTransform, pvalue, @@ -33,8 +31,6 @@ from apache_beam.runners.direct.direct_runner import _GroupAlsoByWindowDoFn from apache_beam.transforms import ptransform from apache_beam.transforms.window import GlobalWindows -from apache_beam.typehints import TypeCheckError -from apache_beam.utils.windowed_value import WindowedValue K = t.TypeVar("K") V = t.TypeVar("V") @@ -81,6 +77,7 @@ def infer_output_type(self, input_type): @typehints.with_input_types(t.Tuple[K, t.Iterable[V]]) @typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) class _GroupAlsoByWindow(ParDo): + """Not used yet...""" def __init__(self, windowing): super(_GroupAlsoByWindow, self).__init__( @@ -98,10 +95,7 @@ class _GroupByKey(PTransform): def expand(self, input_or_inputs): return ( input_or_inputs - # | "ReifyWindows" >> ParDo(GroupByKey.ReifyWindows()) | "GroupByKey" >> _GroupByKeyOnly() - # | "GetValue" >> beam.Map(lambda p: p[1]) - # | "GroupByWindow" >> _GroupAlsoByWindow(input_or_inputs.windowing) ) diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 4f1f02d22e94..86817b0cbe95 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -32,6 +32,8 @@ _Flatten ) +OpInput = t.Union[db.Bag, t.Sequence[db.Bag], None] + @dataclasses.dataclass class DaskBagOp(abc.ABC): @@ -42,17 +44,17 @@ def side_inputs(self): return self.applied.side_inputs @abc.abstractmethod - def apply(self, input_bag: t.Optional[db.Bag]) -> db.Bag: + def apply(self, input_bag: OpInput) -> db.Bag: pass class NoOp(DaskBagOp): - def apply(self, input_bag: t.Optional[db.Bag]) -> db.Bag: + def apply(self, input_bag: OpInput) -> db.Bag: return input_bag class Create(DaskBagOp): - def apply(self, input_bag: t.Optional[db.Bag]) -> db.Bag: + def apply(self, input_bag: OpInput) -> db.Bag: assert input_bag is None, 'Create expects no input!' original_transform = t.cast(_Create, self.applied.transform) items = original_transform.values @@ -60,19 +62,19 @@ def apply(self, input_bag: t.Optional[db.Bag]) -> db.Bag: class ParDo(DaskBagOp): - def apply(self, input_bag: t.Optional[db.Bag]) -> db.Bag: + def apply(self, input_bag: OpInput) -> db.Bag: fn = t.cast(apache_beam.ParDo, self.applied.transform).fn return input_bag.map(fn.process).flatten() class Map(DaskBagOp): - def apply(self, input_bag: t.Optional[db.Bag]) -> db.Bag: + def apply(self, input_bag: OpInput) -> db.Bag: fn = t.cast(apache_beam.Map, self.applied.transform).fn return input_bag.map(fn.process) class GroupByKey(DaskBagOp): - def apply(self, input_bag: t.Optional[db.Bag]) -> db.Bag: + def apply(self, input_bag: OpInput) -> db.Bag: def key(item): return item[0] @@ -84,7 +86,7 @@ def value(item): class Flatten(DaskBagOp): - def apply(self, input_bag: t.Optional[t.Sequence[db.Bag]]) -> db.Bag: + def apply(self, input_bag: OpInput) -> db.Bag: assert type(input_bag) is list, 'Must take a sequence of bags!' return db.concat(input_bag) From 1b58d4cbc485cf6da25677894ec0bef6b3aeebcd Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Fri, 8 Jul 2022 15:48:00 -0700 Subject: [PATCH 012/173] Self-review: Remove TODOs, delete commented out code, other cleanup. --- .../apache_beam/runners/dask/dask_runner.py | 14 ------- .../apache_beam/runners/dask/overrides.py | 37 ++++++++----------- .../runners/dask/transform_evaluator.py | 6 ++- 3 files changed, 19 insertions(+), 38 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index e77d223f178c..088cd227c2e0 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -145,7 +145,6 @@ def run_pipeline(self, pipeline, options): except ImportError: raise ImportError('DaskRunner is not available. Please install apache_beam[dask].') - # TODO(alxr): Actually use this right. dask_options = options.view_as(DaskOptions).get_all_options(drop_default=True) client = ddist.Client(**dask_options) @@ -157,16 +156,3 @@ def run_pipeline(self, pipeline, options): futures = client.compute(list(dask_visitor.bags.values())) return DaskRunnerResult(client, futures) - - # if pipeline: - # pass - # else: - # raise ValueError('Proto or FunctionAPI environments are not supported.') - # if pipeline: - # - # # Flatten / Optimize graph? - # - # # Trigger a traversal of all reachable nodes. - # self.visit_transforms(pipeline, options) - # - # Get API Client? diff --git a/sdks/python/apache_beam/runners/dask/overrides.py b/sdks/python/apache_beam/runners/dask/overrides.py index 3b172535bfe2..008b7d5d92e8 100644 --- a/sdks/python/apache_beam/runners/dask/overrides.py +++ b/sdks/python/apache_beam/runners/dask/overrides.py @@ -18,13 +18,6 @@ import typing as t import apache_beam as beam -from apache_beam import ( - Create, - ParDo, - PTransform, - pvalue, - Windowing, -) from apache_beam import typehints from apache_beam.io.iobase import SourceBase from apache_beam.pipeline import PTransformOverride, AppliedPTransform @@ -37,37 +30,37 @@ @dataclasses.dataclass -class _Create(PTransform): +class _Create(beam.PTransform): values: t.Tuple[t.Any] def expand(self, input_or_inputs): - return pvalue.PCollection.from_(input_or_inputs) + return beam.pvalue.PCollection.from_(input_or_inputs) def get_windowing(self, inputs): # type: (typing.Any) -> Windowing - return Windowing(GlobalWindows()) + return beam.Windowing(GlobalWindows()) @typehints.with_input_types(K) @typehints.with_output_types(K) -class _Reshuffle(PTransform): +class _Reshuffle(beam.PTransform): def expand(self, input_or_inputs): - return pvalue.PCollection.from_(input_or_inputs) + return beam.pvalue.PCollection.from_(input_or_inputs) @dataclasses.dataclass -class _Read(PTransform): +class _Read(beam.PTransform): source: SourceBase def expand(self, input_or_inputs): - return pvalue.PCollection.from_(input_or_inputs) + return beam.pvalue.PCollection.from_(input_or_inputs) @typehints.with_input_types(t.Tuple[K, V]) @typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) -class _GroupByKeyOnly(PTransform): +class _GroupByKeyOnly(beam.PTransform): def expand(self, input_or_inputs): - return pvalue.PCollection.from_(input_or_inputs) + return beam.pvalue.PCollection.from_(input_or_inputs) def infer_output_type(self, input_type): key_type, value_type = typehints.trivial_inference.key_value_types(input_type) @@ -76,7 +69,7 @@ def infer_output_type(self, input_type): @typehints.with_input_types(t.Tuple[K, t.Iterable[V]]) @typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) -class _GroupAlsoByWindow(ParDo): +class _GroupAlsoByWindow(beam.ParDo): """Not used yet...""" def __init__(self, windowing): @@ -85,12 +78,12 @@ def __init__(self, windowing): self.windowing = windowing def expand(self, input_or_inputs): - return pvalue.PCollection.from_(input_or_inputs) + return beam.pvalue.PCollection.from_(input_or_inputs) @typehints.with_input_types(t.Tuple[K, V]) @typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) -class _GroupByKey(PTransform): +class _GroupByKey(beam.PTransform): def expand(self, input_or_inputs): return ( @@ -99,10 +92,10 @@ def expand(self, input_or_inputs): ) -class _Flatten(PTransform): +class _Flatten(beam.PTransform): def expand(self, input_or_inputs): is_bounded = all(pcoll.is_bounded for pcoll in input_or_inputs) - return pvalue.PCollection(self.pipeline, is_bounded=is_bounded) + return beam.pvalue.PCollection(self.pipeline, is_bounded=is_bounded) def dask_overrides() -> t.List[PTransformOverride]: @@ -112,7 +105,7 @@ def matches(self, applied_ptransform: AppliedPTransform) -> bool: def get_replacement_transform_for_applied_ptransform( self, applied_ptransform: AppliedPTransform) -> ptransform.PTransform: - return _Create(t.cast(Create, applied_ptransform.transform).values) + return _Create(t.cast(beam.Create, applied_ptransform.transform).values) class ReshuffleOverride(PTransformOverride): def matches(self, applied_ptransform: AppliedPTransform) -> bool: diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 86817b0cbe95..cada317aa0db 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -14,10 +14,12 @@ # See the License for the specific language governing permissions and # limitations under the License. # -""" +"""Transform Beam PTransforms into Dask Bag operations. -TODO(alxr): Translate ops from https://docs.dask.org/en/latest/bag-api.html. +A minimum set of operation substitutions, to adap Beam's PTransform model +to Dask Bag functions. """ +# TODO(alxr): Translate ops from https://docs.dask.org/en/latest/bag-api.html. import abc import dataclasses import typing as t From 5fe9372ce2722a6a388d4c2f1355d4ea33dacc78 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 9 Jul 2022 00:07:39 -0700 Subject: [PATCH 013/173] First pass at linting rules. --- .../apache_beam/runners/dask/dask_runner.py | 217 ++++++++++-------- .../runners/dask/dask_runner_test.py | 47 ++-- .../apache_beam/runners/dask/overrides.py | 161 ++++++------- .../runners/dask/transform_evaluator.py | 85 ++++--- 4 files changed, 263 insertions(+), 247 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index 088cd227c2e0..6fd1af8ceeb8 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -20,139 +20,152 @@ transformations across processes and workers via Dask distributed's scheduler. """ -import argparse import dataclasses -import typing as t -import dask.distributed +import argparse +import typing as t from apache_beam import pvalue from apache_beam.options.pipeline_options import PipelineOptions -from apache_beam.pipeline import PipelineVisitor, AppliedPTransform +from apache_beam.pipeline import AppliedPTransform +from apache_beam.pipeline import PipelineVisitor from apache_beam.runners.dask.overrides import dask_overrides -from apache_beam.runners.dask.transform_evaluator import TRANSLATIONS, NoOp +from apache_beam.runners.dask.transform_evaluator import TRANSLATIONS +from apache_beam.runners.dask.transform_evaluator import NoOp from apache_beam.runners.direct.direct_runner import BundleBasedDirectRunner -from apache_beam.runners.runner import PipelineResult, PipelineState +from apache_beam.runners.runner import PipelineResult +from apache_beam.runners.runner import PipelineState from apache_beam.utils.interactive_utils import is_in_notebook class DaskOptions(PipelineOptions): - @staticmethod - def _parse_timeout(candidate): - try: - return int(candidate) - except: # noqa - import dask - return dask.config.no_default - - @classmethod - def _add_argparse_args(cls, parser: argparse.ArgumentParser) -> None: - parser.add_argument('--dask_client_address', type=str, default=None, - help='Address of a dask Scheduler server. Will default to a `dask.LocalCluster()`.') - parser.add_argument('--dask_connection_timeout', dest='timeout', type=DaskOptions._parse_timeout, - help='Timeout duration for initial connection to the scheduler.') - parser.add_argument('--dask_scheduler_file', type=str, default=None, - help='Path to a file with scheduler information if available.') - # TODO(alxr): Add options for security. - parser.add_argument('--dask_client_name', dest='name', type=str, default=None, - help='Gives the client a name that will be included in logs generated on the scheduler ' - 'for matters relating to this client.') - parser.add_argument('--dask_connection_limit', dest='connection_limit', type=int, default=512, - help='The number of open comms to maintain at once in the connection pool.') + @staticmethod + def _parse_timeout(candidate): + try: + return int(candidate) + except (TypeError, ValueError): + import dask + return dask.config.no_default + + @classmethod + def _add_argparse_args(cls, parser: argparse.ArgumentParser) -> None: + parser.add_argument('--dask_client_address', type=str, default=None, + help='Address of a dask Scheduler server. Will ' + 'default to a `dask.LocalCluster()`.') + parser.add_argument('--dask_connection_timeout', dest='timeout', + type=DaskOptions._parse_timeout, + help='Timeout duration for initial connection to the ' + 'scheduler.') + parser.add_argument('--dask_scheduler_file', type=str, default=None, + help='Path to a file with scheduler information if ' + 'available.') + # TODO(alxr): Add options for security. + parser.add_argument('--dask_client_name', dest='name', type=str, + default=None, + help='Gives the client a name that will be included ' + 'in logs generated on the scheduler for matters ' + 'relating to this client.') + parser.add_argument('--dask_connection_limit', dest='connection_limit', + type=int, default=512, + help='The number of open comms to maintain at once in ' + 'the connection pool.') @dataclasses.dataclass class DaskRunnerResult(PipelineResult): - client: 'dask.distributed.Client' - futures: t.Sequence['dask.distributed.Future'] - - def __post_init__(self): - super().__init__(PipelineState.RUNNING) - - def wait_until_finish(self, duration=None) -> PipelineState: - try: - if duration is not None: - # Convert milliseconds to seconds - duration /= 1000 - self.client.wait_for_workers(timeout=duration) - self.client.gather(self.futures, errors='raise') - self._state = PipelineState.DONE - except: # pylint: disable=broad-except - self._state = PipelineState.FAILED - raise - return self._state - - def cancel(self) -> PipelineState: - self._state = PipelineState.CANCELLING - self.client.cancel(self.futures) - self._state = PipelineState.CANCELLED - return self._state - - def metrics(self): - # TODO(alxr): Collect and return metrics... - raise NotImplementedError('collecting metrics will come later!') + from dask import distributed + + client: distributed.Client + futures: t.Sequence[distributed.Future] + + def __post_init__(self): + super().__init__(PipelineState.RUNNING) + + def wait_until_finish(self, duration=None) -> PipelineState: + try: + if duration is not None: + # Convert milliseconds to seconds + duration /= 1000 + self.client.wait_for_workers(timeout=duration) + self.client.gather(self.futures, errors='raise') + self._state = PipelineState.DONE + except: # pylint: disable=broad-except + self._state = PipelineState.FAILED + raise + return self._state + + def cancel(self) -> PipelineState: + self._state = PipelineState.CANCELLING + self.client.cancel(self.futures) + self._state = PipelineState.CANCELLED + return self._state + + def metrics(self): + # TODO(alxr): Collect and return metrics... + raise NotImplementedError('collecting metrics will come later!') class DaskRunner(BundleBasedDirectRunner): - """Executes a pipeline on a Dask distributed client.""" - - @staticmethod - def to_dask_bag_visitor() -> 'DaskBagVisitor': - from dask import bag as db + """Executes a pipeline on a Dask distributed client.""" - @dataclasses.dataclass - class DaskBagVisitor(PipelineVisitor): - bags: t.Dict[AppliedPTransform, db.Bag] = dataclasses.field(default_factory=dict) + @staticmethod + def to_dask_bag_visitor() -> PipelineVisitor: + from dask import bag as db - def visit_transform(self, transform_node: AppliedPTransform) -> None: - op_class = TRANSLATIONS.get(transform_node.transform.__class__, NoOp) - op = op_class(transform_node) + @dataclasses.dataclass + class DaskBagVisitor(PipelineVisitor): + bags: t.Dict[AppliedPTransform, db.Bag] = dataclasses.field( + default_factory=dict) - inputs = list(transform_node.inputs) - if inputs: - bag_inputs = [] - for input_value in inputs: - if isinstance(input_value, pvalue.PBegin): - bag_inputs.append(None) + def visit_transform(self, transform_node: AppliedPTransform) -> None: + op_class = TRANSLATIONS.get(transform_node.transform.__class__, NoOp) + op = op_class(transform_node) - prev_op = input_value.producer - if prev_op in self.bags: - bag_inputs.append(self.bags[prev_op]) + inputs = list(transform_node.inputs) + if inputs: + bag_inputs = [] + for input_value in inputs: + if isinstance(input_value, pvalue.PBegin): + bag_inputs.append(None) - if len(bag_inputs) == 1: - self.bags[transform_node] = op.apply(bag_inputs[0]) - else: - self.bags[transform_node] = op.apply(bag_inputs) + prev_op = input_value.producer + if prev_op in self.bags: + bag_inputs.append(self.bags[prev_op]) - else: - self.bags[transform_node] = op.apply(None) + if len(bag_inputs) == 1: + self.bags[transform_node] = op.apply(bag_inputs[0]) + else: + self.bags[transform_node] = op.apply(bag_inputs) - return DaskBagVisitor() + else: + self.bags[transform_node] = op.apply(None) - @staticmethod - def is_fnapi_compatible(): - return False + return DaskBagVisitor() - def run_pipeline(self, pipeline, options): - # TODO(alxr): Create interactive notebook support. - if is_in_notebook(): - raise NotImplementedError('interactive support will come later!') + @staticmethod + def is_fnapi_compatible(): + return False - try: - import dask.bag as db - import dask.distributed as ddist - except ImportError: - raise ImportError('DaskRunner is not available. Please install apache_beam[dask].') + def run_pipeline(self, pipeline, options): + # TODO(alxr): Create interactive notebook support. + if is_in_notebook(): + raise NotImplementedError('interactive support will come later!') - dask_options = options.view_as(DaskOptions).get_all_options(drop_default=True) - client = ddist.Client(**dask_options) + try: + import dask.distributed as ddist + except ImportError: + raise ImportError( + 'DaskRunner is not available. Please install apache_beam[dask].') - pipeline.replace_all(dask_overrides()) + dask_options = options.view_as(DaskOptions).get_all_options( + drop_default=True) + client = ddist.Client(**dask_options) - dask_visitor = self.to_dask_bag_visitor() - pipeline.visit(dask_visitor) + pipeline.replace_all(dask_overrides()) - futures = client.compute(list(dask_visitor.bags.values())) + dask_visitor = self.to_dask_bag_visitor() + pipeline.visit(dask_visitor) - return DaskRunnerResult(client, futures) + futures = client.compute(list(dask_visitor.bags.values())) + return DaskRunnerResult(client, futures) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index c731a8dae51d..e5a38dbbd83e 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -19,38 +19,39 @@ import apache_beam as beam from apache_beam.runners.dask.dask_runner import DaskRunner from apache_beam.testing import test_pipeline -from apache_beam.testing.util import assert_that, equal_to +from apache_beam.testing.util import assert_that +from apache_beam.testing.util import equal_to class DaskRunnerRunPipelineTest(unittest.TestCase): - """Test class used to introspect the dask runner via a debugger.""" + """Test class used to introspect the dask runner via a debugger.""" - def setUp(self) -> None: - self.pipeline = test_pipeline.TestPipeline(runner=DaskRunner()) + def setUp(self) -> None: + self.pipeline = test_pipeline.TestPipeline(runner=DaskRunner()) - def test_create(self): - with self.pipeline as p: - pcoll = p | beam.Create([1]) - assert_that(pcoll, equal_to([1])) + def test_create(self): + with self.pipeline as p: + pcoll = p | beam.Create([1]) + assert_that(pcoll, equal_to([1])) - def test_create_and_map(self): - def double(x): - return x * 2 + def test_create_and_map(self): + def double(x): + return x * 2 - with self.pipeline as p: - pcoll = p | beam.Create([1]) | beam.Map(double) - assert_that(pcoll, equal_to([2])) + with self.pipeline as p: + pcoll = p | beam.Create([1]) | beam.Map(double) + assert_that(pcoll, equal_to([2])) - def test_create_map_and_groupby(self): - def double(x): - return x * 2, x + def test_create_map_and_groupby(self): + def double(x): + return x * 2, x - with self.pipeline as p: - pcoll = p | beam.Create([1]) | beam.Map(double) | beam.GroupByKey() - assert_that(pcoll, equal_to([ - (2, [1]) - ])) + with self.pipeline as p: + pcoll = p | beam.Create([1]) | beam.Map(double) | beam.GroupByKey() + assert_that(pcoll, equal_to([ + (2, [1]) + ])) if __name__ == '__main__': - unittest.main() + unittest.main() diff --git a/sdks/python/apache_beam/runners/dask/overrides.py b/sdks/python/apache_beam/runners/dask/overrides.py index 008b7d5d92e8..f06b6de6a10e 100644 --- a/sdks/python/apache_beam/runners/dask/overrides.py +++ b/sdks/python/apache_beam/runners/dask/overrides.py @@ -15,12 +15,14 @@ # limitations under the License. # import dataclasses + import typing as t import apache_beam as beam from apache_beam import typehints from apache_beam.io.iobase import SourceBase -from apache_beam.pipeline import PTransformOverride, AppliedPTransform +from apache_beam.pipeline import AppliedPTransform +from apache_beam.pipeline import PTransformOverride from apache_beam.runners.direct.direct_runner import _GroupAlsoByWindowDoFn from apache_beam.transforms import ptransform from apache_beam.transforms.window import GlobalWindows @@ -31,118 +33,119 @@ @dataclasses.dataclass class _Create(beam.PTransform): - values: t.Tuple[t.Any] + values: t.Tuple[t.Any] - def expand(self, input_or_inputs): - return beam.pvalue.PCollection.from_(input_or_inputs) + def expand(self, input_or_inputs): + return beam.pvalue.PCollection.from_(input_or_inputs) - def get_windowing(self, inputs): - # type: (typing.Any) -> Windowing - return beam.Windowing(GlobalWindows()) + def get_windowing(self, inputs: t.Any) -> beam.Windowing: + return beam.Windowing(GlobalWindows()) @typehints.with_input_types(K) @typehints.with_output_types(K) class _Reshuffle(beam.PTransform): - def expand(self, input_or_inputs): - return beam.pvalue.PCollection.from_(input_or_inputs) + def expand(self, input_or_inputs): + return beam.pvalue.PCollection.from_(input_or_inputs) @dataclasses.dataclass class _Read(beam.PTransform): - source: SourceBase + source: SourceBase - def expand(self, input_or_inputs): - return beam.pvalue.PCollection.from_(input_or_inputs) + def expand(self, input_or_inputs): + return beam.pvalue.PCollection.from_(input_or_inputs) @typehints.with_input_types(t.Tuple[K, V]) @typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) class _GroupByKeyOnly(beam.PTransform): - def expand(self, input_or_inputs): - return beam.pvalue.PCollection.from_(input_or_inputs) + def expand(self, input_or_inputs): + return beam.pvalue.PCollection.from_(input_or_inputs) + + def infer_output_type(self, input_type): - def infer_output_type(self, input_type): - key_type, value_type = typehints.trivial_inference.key_value_types(input_type) - return typehints.KV[key_type, typehints.Iterable[value_type]] + key_type, value_type = typehints.trivial_inference.key_value_types( + input_type + ) + return typehints.KV[key_type, typehints.Iterable[value_type]] @typehints.with_input_types(t.Tuple[K, t.Iterable[V]]) @typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) class _GroupAlsoByWindow(beam.ParDo): - """Not used yet...""" + """Not used yet...""" - def __init__(self, windowing): - super(_GroupAlsoByWindow, self).__init__( - _GroupAlsoByWindowDoFn(windowing)) - self.windowing = windowing + def __init__(self, windowing): + super().__init__(_GroupAlsoByWindowDoFn(windowing)) + self.windowing = windowing - def expand(self, input_or_inputs): - return beam.pvalue.PCollection.from_(input_or_inputs) + def expand(self, input_or_inputs): + return beam.pvalue.PCollection.from_(input_or_inputs) @typehints.with_input_types(t.Tuple[K, V]) @typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) class _GroupByKey(beam.PTransform): - def expand(self, input_or_inputs): - return ( - input_or_inputs - | "GroupByKey" >> _GroupByKeyOnly() - ) + def expand(self, input_or_inputs): + return ( + input_or_inputs + | "GroupByKey" >> _GroupByKeyOnly() + ) class _Flatten(beam.PTransform): - def expand(self, input_or_inputs): - is_bounded = all(pcoll.is_bounded for pcoll in input_or_inputs) - return beam.pvalue.PCollection(self.pipeline, is_bounded=is_bounded) + def expand(self, input_or_inputs): + is_bounded = all(pcoll.is_bounded for pcoll in input_or_inputs) + return beam.pvalue.PCollection(self.pipeline, is_bounded=is_bounded) def dask_overrides() -> t.List[PTransformOverride]: - class CreateOverride(PTransformOverride): - def matches(self, applied_ptransform: AppliedPTransform) -> bool: - return applied_ptransform.transform.__class__ == beam.Create - - def get_replacement_transform_for_applied_ptransform( - self, applied_ptransform: AppliedPTransform) -> ptransform.PTransform: - return _Create(t.cast(beam.Create, applied_ptransform.transform).values) - - class ReshuffleOverride(PTransformOverride): - def matches(self, applied_ptransform: AppliedPTransform) -> bool: - return applied_ptransform.transform.__class__ == beam.Reshuffle - - def get_replacement_transform_for_applied_ptransform( - self, applied_ptransform: AppliedPTransform) -> ptransform.PTransform: - return _Reshuffle() - - class ReadOverride(PTransformOverride): - def matches(self, applied_ptransform: AppliedPTransform) -> bool: - return applied_ptransform.transform.__class__ == beam.io.Read - - def get_replacement_transform_for_applied_ptransform( - self, applied_ptransform: AppliedPTransform) -> ptransform.PTransform: - return _Read(t.cast(beam.io.Read, applied_ptransform.transform).source) - - class GroupByKeyOverride(PTransformOverride): - def matches(self, applied_ptransform: AppliedPTransform) -> bool: - return applied_ptransform.transform.__class__ == beam.GroupByKey - - def get_replacement_transform_for_applied_ptransform( - self, applied_ptransform: AppliedPTransform) -> ptransform.PTransform: - return _GroupByKey() - - class FlattenOverride(PTransformOverride): - def matches(self, applied_ptransform: AppliedPTransform) -> bool: - return applied_ptransform.transform.__class__ == beam.Flatten - - def get_replacement_transform_for_applied_ptransform( - self, applied_ptransform: AppliedPTransform) -> ptransform.PTransform: - return _Flatten() - - return [ - CreateOverride(), - ReshuffleOverride(), - ReadOverride(), - GroupByKeyOverride(), - FlattenOverride(), - ] + class CreateOverride(PTransformOverride): + def matches(self, applied_ptransform: AppliedPTransform) -> bool: + return applied_ptransform.transform.__class__ == beam.Create + + def get_replacement_transform_for_applied_ptransform( + self, applied_ptransform: AppliedPTransform) -> ptransform.PTransform: + return _Create(t.cast(beam.Create, applied_ptransform.transform).values) + + class ReshuffleOverride(PTransformOverride): + def matches(self, applied_ptransform: AppliedPTransform) -> bool: + return applied_ptransform.transform.__class__ == beam.Reshuffle + + def get_replacement_transform_for_applied_ptransform( + self, applied_ptransform: AppliedPTransform) -> ptransform.PTransform: + return _Reshuffle() + + class ReadOverride(PTransformOverride): + def matches(self, applied_ptransform: AppliedPTransform) -> bool: + return applied_ptransform.transform.__class__ == beam.io.Read + + def get_replacement_transform_for_applied_ptransform( + self, applied_ptransform: AppliedPTransform) -> ptransform.PTransform: + return _Read(t.cast(beam.io.Read, applied_ptransform.transform).source) + + class GroupByKeyOverride(PTransformOverride): + def matches(self, applied_ptransform: AppliedPTransform) -> bool: + return applied_ptransform.transform.__class__ == beam.GroupByKey + + def get_replacement_transform_for_applied_ptransform( + self, applied_ptransform: AppliedPTransform) -> ptransform.PTransform: + return _GroupByKey() + + class FlattenOverride(PTransformOverride): + def matches(self, applied_ptransform: AppliedPTransform) -> bool: + return applied_ptransform.transform.__class__ == beam.Flatten + + def get_replacement_transform_for_applied_ptransform( + self, applied_ptransform: AppliedPTransform) -> ptransform.PTransform: + return _Flatten() + + return [ + CreateOverride(), + ReshuffleOverride(), + ReadOverride(), + GroupByKeyOverride(), + FlattenOverride(), + ] diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index cada317aa0db..ae58a8bae414 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -18,85 +18,84 @@ A minimum set of operation substitutions, to adap Beam's PTransform model to Dask Bag functions. + +TODO(alxr): Translate ops from https://docs.dask.org/en/latest/bag-api.html. """ -# TODO(alxr): Translate ops from https://docs.dask.org/en/latest/bag-api.html. -import abc import dataclasses -import typing as t +import abc import dask.bag as db +import typing as t import apache_beam from apache_beam.pipeline import AppliedPTransform -from apache_beam.runners.dask.overrides import ( - _Create, - _GroupByKeyOnly, - _Flatten -) +from apache_beam.runners.dask.overrides import _Create +from apache_beam.runners.dask.overrides import _Flatten +from apache_beam.runners.dask.overrides import _GroupByKeyOnly OpInput = t.Union[db.Bag, t.Sequence[db.Bag], None] @dataclasses.dataclass class DaskBagOp(abc.ABC): - applied: AppliedPTransform + applied: AppliedPTransform - @property - def side_inputs(self): - return self.applied.side_inputs + @property + def side_inputs(self): + return self.applied.side_inputs - @abc.abstractmethod - def apply(self, input_bag: OpInput) -> db.Bag: - pass + @abc.abstractmethod + def apply(self, input_bag: OpInput) -> db.Bag: + pass class NoOp(DaskBagOp): - def apply(self, input_bag: OpInput) -> db.Bag: - return input_bag + def apply(self, input_bag: OpInput) -> db.Bag: + return input_bag class Create(DaskBagOp): - def apply(self, input_bag: OpInput) -> db.Bag: - assert input_bag is None, 'Create expects no input!' - original_transform = t.cast(_Create, self.applied.transform) - items = original_transform.values - return db.from_sequence(items) + def apply(self, input_bag: OpInput) -> db.Bag: + assert input_bag is None, 'Create expects no input!' + original_transform = t.cast(_Create, self.applied.transform) + items = original_transform.values + return db.from_sequence(items) class ParDo(DaskBagOp): - def apply(self, input_bag: OpInput) -> db.Bag: - fn = t.cast(apache_beam.ParDo, self.applied.transform).fn - return input_bag.map(fn.process).flatten() + def apply(self, input_bag: OpInput) -> db.Bag: + fn = t.cast(apache_beam.ParDo, self.applied.transform).fn + return input_bag.map(fn.process).flatten() class Map(DaskBagOp): - def apply(self, input_bag: OpInput) -> db.Bag: - fn = t.cast(apache_beam.Map, self.applied.transform).fn - return input_bag.map(fn.process) + def apply(self, input_bag: OpInput) -> db.Bag: + fn = t.cast(apache_beam.Map, self.applied.transform).fn + return input_bag.map(fn.process) class GroupByKey(DaskBagOp): - def apply(self, input_bag: OpInput) -> db.Bag: - def key(item): - return item[0] + def apply(self, input_bag: OpInput) -> db.Bag: + def key(item): + return item[0] - def value(item): - k, v = item - return k, [elm[1] for elm in v] + def value(item): + k, v = item + return k, [elm[1] for elm in v] - return input_bag.groupby(key).map(value) + return input_bag.groupby(key).map(value) class Flatten(DaskBagOp): - def apply(self, input_bag: OpInput) -> db.Bag: - assert type(input_bag) is list, 'Must take a sequence of bags!' - return db.concat(input_bag) + def apply(self, input_bag: OpInput) -> db.Bag: + assert type(input_bag) is list, 'Must take a sequence of bags!' + return db.concat(input_bag) TRANSLATIONS = { - _Create: Create, - apache_beam.ParDo: ParDo, - apache_beam.Map: Map, - _GroupByKeyOnly: GroupByKey, - _Flatten: Flatten, + _Create: Create, + apache_beam.ParDo: ParDo, + apache_beam.Map: Map, + _GroupByKeyOnly: GroupByKey, + _Flatten: Flatten, } From b98330ecd955fc667dec4e7360e263335cfd7b18 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 9 Jul 2022 00:18:58 -0700 Subject: [PATCH 014/173] WIP, include dask dependencies + test setup. --- sdks/python/setup.py | 3 ++- sdks/python/tox.ini | 6 +++++- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/sdks/python/setup.py b/sdks/python/setup.py index 8017c1b692a7..f1c5f4f6e7c9 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -326,7 +326,8 @@ def get_portability_package_data(): 'azure-storage-blob >=12.3.2', 'azure-core >=1.7.0', ], - 'dataframe': ['pandas>=1.0,<1.5'] + 'dataframe': ['pandas>=1.0,<1.5'], + 'dask': ['dask[distributed] >= 2022.6'] }, zip_safe=False, # PyPI package information. diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini index 592376a01449..b9693c2b8fb5 100644 --- a/sdks/python/tox.ini +++ b/sdks/python/tox.ini @@ -17,7 +17,7 @@ [tox] # new environments will be excluded by default unless explicitly added to envlist. -envlist = py37,py38,py39,py37-{cloud,cython,lint,mypy},py38-{cloud,cython,docs,cloudcoverage},py39-{cloud,cython},whitespacelint +envlist = py37,py38,py39,py37-{cloud,cython,lint,mypy,dask},py38-{cloud,cython,docs,cloudcoverage,dask},py39-{cloud,cython,dask},whitespacelint toxworkdir = {toxinidir}/target/{env:ENV_NAME:.tox} [pycodestyle] @@ -92,6 +92,10 @@ extras = test,gcp,interactive,dataframe,aws,azure commands = {toxinidir}/scripts/run_pytest.sh {envname} "{posargs}" +[testenv:py{37,38,39}-dask] +extras = test,dask +commands = + {toxinidir}/scripts/run_pytest.sh {envname} "{posargs}" [testenv:py38-cloudcoverage] deps = codecov From e48780a8c0c6552e05fe8e542ac70682445a1822 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 9 Jul 2022 00:34:11 -0700 Subject: [PATCH 015/173] WIP: maybe better dask deps? --- sdks/python/setup.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/sdks/python/setup.py b/sdks/python/setup.py index f1c5f4f6e7c9..3153cae8a5a9 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -327,7 +327,10 @@ def get_portability_package_data(): 'azure-core >=1.7.0', ], 'dataframe': ['pandas>=1.0,<1.5'], - 'dask': ['dask[distributed] >= 2022.6'] + 'dask': [ + 'dask >= 2022.6' + 'distributed >= 2022.6' + ], }, zip_safe=False, # PyPI package information. From 0dc9e2397022263821d3a03490fc8452162d161a Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sun, 10 Jul 2022 16:41:59 -0700 Subject: [PATCH 016/173] Skip dask tests depending on successful import. --- sdks/python/apache_beam/runners/dask/dask_runner_test.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index e5a38dbbd83e..208ce77bace1 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -22,6 +22,11 @@ from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to +try: + import dask +except (ImportError, ModuleNotFoundError): + raise unittest.SkipTest('Dask must be installed to run tests.') + class DaskRunnerRunPipelineTest(unittest.TestCase): """Test class used to introspect the dask runner via a debugger.""" From 326d3a38e2ad6b36e964c2725b6a859b15219501 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sun, 10 Jul 2022 17:30:52 -0700 Subject: [PATCH 017/173] Fixed setup.py (missing `,`). --- sdks/python/setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/setup.py b/sdks/python/setup.py index 3153cae8a5a9..e98214d3b431 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -328,7 +328,7 @@ def get_portability_package_data(): ], 'dataframe': ['pandas>=1.0,<1.5'], 'dask': [ - 'dask >= 2022.6' + 'dask >= 2022.6', 'distributed >= 2022.6' ], }, From b4cc408afc0f4618596ee46415f0d34a83e0f749 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sun, 10 Jul 2022 17:31:13 -0700 Subject: [PATCH 018/173] Added an additional comma. --- sdks/python/setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/setup.py b/sdks/python/setup.py index e98214d3b431..a9b94f6e0ab3 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -329,7 +329,7 @@ def get_portability_package_data(): 'dataframe': ['pandas>=1.0,<1.5'], 'dask': [ 'dask >= 2022.6', - 'distributed >= 2022.6' + 'distributed >= 2022.6', ], }, zip_safe=False, From 40a6ebebdcac84626c8fc6755dac6e016dbdf4d6 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sun, 10 Jul 2022 20:14:03 -0700 Subject: [PATCH 019/173] Moved skipping logic to be above dask import. --- .../apache_beam/runners/dask/dask_runner_test.py | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index 208ce77bace1..7e9041d7a67e 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -16,23 +16,24 @@ # import unittest +try: + import dask +except (ImportError, ModuleNotFoundError): + raise unittest.SkipTest('Dask must be installed to run tests.') + import apache_beam as beam from apache_beam.runners.dask.dask_runner import DaskRunner from apache_beam.testing import test_pipeline from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to -try: - import dask -except (ImportError, ModuleNotFoundError): - raise unittest.SkipTest('Dask must be installed to run tests.') class DaskRunnerRunPipelineTest(unittest.TestCase): """Test class used to introspect the dask runner via a debugger.""" def setUp(self) -> None: - self.pipeline = test_pipeline.TestPipeline(runner=DaskRunner()) + self.pipeline = test_pipeline.TestPipeline(runner=DaskRunner()) def test_create(self): with self.pipeline as p: From 3c4204d7d20bc6aab5ca80febd6cbd4614d7c4fd Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Mon, 5 Sep 2022 15:03:34 -0700 Subject: [PATCH 020/173] Fix lint issues with dask runner tests. --- sdks/python/apache_beam/runners/dask/dask_runner_test.py | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index 7e9041d7a67e..53fa3fff4838 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -17,7 +17,7 @@ import unittest try: - import dask + import dask # pylint: disable=unused-import except (ImportError, ModuleNotFoundError): raise unittest.SkipTest('Dask must be installed to run tests.') @@ -28,12 +28,11 @@ from apache_beam.testing.util import equal_to - class DaskRunnerRunPipelineTest(unittest.TestCase): """Test class used to introspect the dask runner via a debugger.""" def setUp(self) -> None: - self.pipeline = test_pipeline.TestPipeline(runner=DaskRunner()) + self.pipeline = test_pipeline.TestPipeline(runner=DaskRunner()) def test_create(self): with self.pipeline as p: From 41623ecfe61726c06d24a7f3e56b941f866088c1 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Mon, 19 Sep 2022 22:32:30 -0700 Subject: [PATCH 021/173] Adding destination for client address. --- sdks/python/apache_beam/runners/dask/dask_runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index 6fd1af8ceeb8..74915c4e254c 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -50,7 +50,7 @@ def _parse_timeout(candidate): @classmethod def _add_argparse_args(cls, parser: argparse.ArgumentParser) -> None: - parser.add_argument('--dask_client_address', type=str, default=None, + parser.add_argument('--dask_client_address', dest='address', type=str, default=None, help='Address of a dask Scheduler server. Will ' 'default to a `dask.LocalCluster()`.') parser.add_argument('--dask_connection_timeout', dest='timeout', From 676d75226c3f517083baef5dab4a52e5cde0f1a0 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Wed, 21 Sep 2022 14:29:37 -0700 Subject: [PATCH 022/173] Changing to async produces a timeout error instead of stuck in infinite loop. --- sdks/python/apache_beam/runners/dask/dask_runner.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index 74915c4e254c..ed711ae00657 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -88,11 +88,13 @@ def wait_until_finish(self, duration=None) -> PipelineState: # Convert milliseconds to seconds duration /= 1000 self.client.wait_for_workers(timeout=duration) - self.client.gather(self.futures, errors='raise') + self.client.gather(self.futures, errors='raise', asynchronous=True) self._state = PipelineState.DONE except: # pylint: disable=broad-except self._state = PipelineState.FAILED raise + # finally: + # self.client.close(timeout=duration) return self._state def cancel(self) -> PipelineState: From 09365f628dc86832190efd03a308ba177755bf13 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Thu, 22 Sep 2022 08:59:42 -0700 Subject: [PATCH 023/173] Close client during `wait_until_finish`; rm async. --- sdks/python/apache_beam/runners/dask/dask_runner.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index ed711ae00657..5b4c297052c0 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -88,13 +88,14 @@ def wait_until_finish(self, duration=None) -> PipelineState: # Convert milliseconds to seconds duration /= 1000 self.client.wait_for_workers(timeout=duration) - self.client.gather(self.futures, errors='raise', asynchronous=True) + self.client.gather(self.futures, errors='raise') self._state = PipelineState.DONE except: # pylint: disable=broad-except self._state = PipelineState.FAILED raise - # finally: - # self.client.close(timeout=duration) + finally: + self.client.close() + self.client.cluster.close() return self._state def cancel(self) -> PipelineState: From c6ba4ba9fc05e9409a121bac31eb02bab3c4ab0b Mon Sep 17 00:00:00 2001 From: Pablo E Date: Wed, 28 Sep 2022 12:03:02 -0700 Subject: [PATCH 024/173] Revert "Close client during `wait_until_finish`; rm async." This reverts commit 09365f628dc86832190efd03a308ba177755bf13. --- sdks/python/apache_beam/runners/dask/dask_runner.py | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index 5b4c297052c0..ed711ae00657 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -88,14 +88,13 @@ def wait_until_finish(self, duration=None) -> PipelineState: # Convert milliseconds to seconds duration /= 1000 self.client.wait_for_workers(timeout=duration) - self.client.gather(self.futures, errors='raise') + self.client.gather(self.futures, errors='raise', asynchronous=True) self._state = PipelineState.DONE except: # pylint: disable=broad-except self._state = PipelineState.FAILED raise - finally: - self.client.close() - self.client.cluster.close() + # finally: + # self.client.close(timeout=duration) return self._state def cancel(self) -> PipelineState: From a325356c2315ebea9878a9198a0d8f0ae013b6c7 Mon Sep 17 00:00:00 2001 From: Pablo E Date: Wed, 28 Sep 2022 12:03:07 -0700 Subject: [PATCH 025/173] Revert "Changing to async produces a timeout error instead of stuck in infinite loop." This reverts commit 676d75226c3f517083baef5dab4a52e5cde0f1a0. --- sdks/python/apache_beam/runners/dask/dask_runner.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index ed711ae00657..74915c4e254c 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -88,13 +88,11 @@ def wait_until_finish(self, duration=None) -> PipelineState: # Convert milliseconds to seconds duration /= 1000 self.client.wait_for_workers(timeout=duration) - self.client.gather(self.futures, errors='raise', asynchronous=True) + self.client.gather(self.futures, errors='raise') self._state = PipelineState.DONE except: # pylint: disable=broad-except self._state = PipelineState.FAILED raise - # finally: - # self.client.close(timeout=duration) return self._state def cancel(self) -> PipelineState: From ea131253e7c4fdcb39dcb7fb676c6915a39773a3 Mon Sep 17 00:00:00 2001 From: Pablo E Date: Wed, 28 Sep 2022 12:04:21 -0700 Subject: [PATCH 026/173] Adding -dask tox targets onto the gradle build --- sdks/python/test-suites/tox/common.gradle | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sdks/python/test-suites/tox/common.gradle b/sdks/python/test-suites/tox/common.gradle index 99afc1d72557..61802ac9c45e 100644 --- a/sdks/python/test-suites/tox/common.gradle +++ b/sdks/python/test-suites/tox/common.gradle @@ -24,6 +24,9 @@ test.dependsOn "testPython${pythonVersionSuffix}" toxTask "testPy${pythonVersionSuffix}Cloud", "py${pythonVersionSuffix}-cloud" test.dependsOn "testPy${pythonVersionSuffix}Cloud" +toxTask "testPy${pythonVersionSuffix}Dask", "py${pythonVersionSuffix}-dask" +test.dependsOn "testPy${pythonVersionSuffix}Dask" + toxTask "testPy${pythonVersionSuffix}Cython", "py${pythonVersionSuffix}-cython" test.dependsOn "testPy${pythonVersionSuffix}Cython" From f855ffc5d6bcd3c8064cccb214b590a9951d84d0 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 1 Oct 2022 21:24:14 -0400 Subject: [PATCH 027/173] Supporting side-inputs for ParDo. --- sdks/python/apache_beam/runners/dask/dask_runner_test.py | 9 +++++++++ .../apache_beam/runners/dask/transform_evaluator.py | 5 +++-- 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index 53fa3fff4838..bd9253e6bc2b 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -57,6 +57,15 @@ def double(x): (2, [1]) ])) + def test_map_with_side_inputs(self): + def mult_by(x, y=2): + return x * y + + with self.pipeline as p: + pcoll = p | beam.Create([1]) | beam.Map(mult_by, y=3) + assert_that(pcoll, equal_to([3])) + + if __name__ == '__main__': unittest.main() diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index ae58a8bae414..92c5922c344a 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -26,6 +26,7 @@ import abc import dask.bag as db import typing as t +import functools import apache_beam from apache_beam.pipeline import AppliedPTransform @@ -65,13 +66,13 @@ def apply(self, input_bag: OpInput) -> db.Bag: class ParDo(DaskBagOp): def apply(self, input_bag: OpInput) -> db.Bag: fn = t.cast(apache_beam.ParDo, self.applied.transform).fn - return input_bag.map(fn.process).flatten() + return input_bag.map(fn.process, *self.side_inputs).flatten() class Map(DaskBagOp): def apply(self, input_bag: OpInput) -> db.Bag: fn = t.cast(apache_beam.Map, self.applied.transform).fn - return input_bag.map(fn.process) + return input_bag.map(fn.process, *self.side_inputs) class GroupByKey(DaskBagOp): From 173d79bf3598aa397c390f6a2f1499397be19567 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 1 Oct 2022 21:27:03 -0400 Subject: [PATCH 028/173] wip - added print stmt. --- sdks/python/apache_beam/runners/dask/transform_evaluator.py | 1 + 1 file changed, 1 insertion(+) diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 92c5922c344a..2ae7caa130f8 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -72,6 +72,7 @@ def apply(self, input_bag: OpInput) -> db.Bag: class Map(DaskBagOp): def apply(self, input_bag: OpInput) -> db.Bag: fn = t.cast(apache_beam.Map, self.applied.transform).fn + print(self.side_inputs) return input_bag.map(fn.process, *self.side_inputs) From dd2d15c95233634a32db9c2910da5df6257b91c1 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 1 Oct 2022 21:29:57 -0400 Subject: [PATCH 029/173] wip - prove side inputs is set. --- sdks/python/apache_beam/runners/dask/transform_evaluator.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 2ae7caa130f8..5d70a871fc40 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -72,7 +72,7 @@ def apply(self, input_bag: OpInput) -> db.Bag: class Map(DaskBagOp): def apply(self, input_bag: OpInput) -> db.Bag: fn = t.cast(apache_beam.Map, self.applied.transform).fn - print(self.side_inputs) + assert self.side_inputs is (3,) return input_bag.map(fn.process, *self.side_inputs) From 8756618859b9db79f0c27cde217103a363b32ea3 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 1 Oct 2022 21:31:13 -0400 Subject: [PATCH 030/173] wip - prove side inputs is set in Pardo. --- sdks/python/apache_beam/runners/dask/transform_evaluator.py | 1 + 1 file changed, 1 insertion(+) diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 5d70a871fc40..7c24f180aef0 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -66,6 +66,7 @@ def apply(self, input_bag: OpInput) -> db.Bag: class ParDo(DaskBagOp): def apply(self, input_bag: OpInput) -> db.Bag: fn = t.cast(apache_beam.ParDo, self.applied.transform).fn + assert self.side_inputs is (3,) return input_bag.map(fn.process, *self.side_inputs).flatten() From 8380d7bf92d464c1c9042f417de9302ac969b57b Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 1 Oct 2022 21:32:27 -0400 Subject: [PATCH 031/173] wip - rm asserts, add print --- sdks/python/apache_beam/runners/dask/transform_evaluator.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 7c24f180aef0..408bc38cce52 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -66,14 +66,13 @@ def apply(self, input_bag: OpInput) -> db.Bag: class ParDo(DaskBagOp): def apply(self, input_bag: OpInput) -> db.Bag: fn = t.cast(apache_beam.ParDo, self.applied.transform).fn - assert self.side_inputs is (3,) + print(self.side_inputs) return input_bag.map(fn.process, *self.side_inputs).flatten() class Map(DaskBagOp): def apply(self, input_bag: OpInput) -> db.Bag: fn = t.cast(apache_beam.Map, self.applied.transform).fn - assert self.side_inputs is (3,) return input_bag.map(fn.process, *self.side_inputs) From b908dc3b697ead907b4c82c79bd4057fc0193784 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 1 Oct 2022 21:41:05 -0400 Subject: [PATCH 032/173] wip - adding named inputs... --- .../apache_beam/runners/dask/transform_evaluator.py | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 408bc38cce52..50f4ba51279f 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -45,6 +45,10 @@ class DaskBagOp(abc.ABC): def side_inputs(self): return self.applied.side_inputs + @property + def named_inputs(self): + return self.applied.named_inputs() + @abc.abstractmethod def apply(self, input_bag: OpInput) -> db.Bag: pass @@ -67,13 +71,14 @@ class ParDo(DaskBagOp): def apply(self, input_bag: OpInput) -> db.Bag: fn = t.cast(apache_beam.ParDo, self.applied.transform).fn print(self.side_inputs) - return input_bag.map(fn.process, *self.side_inputs).flatten() + print(self.named_inputs) + return input_bag.map(fn.process, *self.side_inputs, **self.named_inputs).flatten() class Map(DaskBagOp): def apply(self, input_bag: OpInput) -> db.Bag: fn = t.cast(apache_beam.Map, self.applied.transform).fn - return input_bag.map(fn.process, *self.side_inputs) + return input_bag.map(fn.process, *self.side_inputs, **self.named_inputs) class GroupByKey(DaskBagOp): From f444b1e2afcfb7003005473ddb5ca0f2a7cb6cc8 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 1 Oct 2022 21:55:06 -0400 Subject: [PATCH 033/173] Experiments: non-named side inputs + del `None` in named inputs. --- sdks/python/apache_beam/runners/dask/dask_runner_test.py | 2 +- sdks/python/apache_beam/runners/dask/transform_evaluator.py | 6 ++++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index bd9253e6bc2b..6e69e90938a9 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -62,7 +62,7 @@ def mult_by(x, y=2): return x * y with self.pipeline as p: - pcoll = p | beam.Create([1]) | beam.Map(mult_by, y=3) + pcoll = p | beam.Create([1]) | beam.Map(mult_by, 3) assert_that(pcoll, equal_to([3])) diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 50f4ba51279f..b77e50e57c1e 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -45,9 +45,11 @@ class DaskBagOp(abc.ABC): def side_inputs(self): return self.applied.side_inputs - @property + @functools.cached_property def named_inputs(self): - return self.applied.named_inputs() + named_inputs = self.applied.named_inputs() + del named_inputs[None] + return named_inputs @abc.abstractmethod def apply(self, input_bag: OpInput) -> db.Bag: From 174d6fd4558e4b3dd3319f6d78077543851a716d Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 1 Oct 2022 21:56:35 -0400 Subject: [PATCH 034/173] None --> 'None' --- sdks/python/apache_beam/runners/dask/transform_evaluator.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index b77e50e57c1e..3f4676c8f3e2 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -48,7 +48,7 @@ def side_inputs(self): @functools.cached_property def named_inputs(self): named_inputs = self.applied.named_inputs() - del named_inputs[None] + del named_inputs['None'] return named_inputs @abc.abstractmethod From 60b063a7e231d1e07e150ee00be82dc659711222 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 1 Oct 2022 21:59:01 -0400 Subject: [PATCH 035/173] No default side input. --- sdks/python/apache_beam/runners/dask/dask_runner_test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index 6e69e90938a9..ecf7e920121f 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -58,7 +58,7 @@ def double(x): ])) def test_map_with_side_inputs(self): - def mult_by(x, y=2): + def mult_by(x, y): return x * y with self.pipeline as p: From 90ee474207e58f8225d2ab0c4b5b25e81d03c96b Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 1 Oct 2022 22:06:36 -0400 Subject: [PATCH 036/173] Pass along args + kwargs. --- .../runners/dask/dask_runner_test.py | 7 +++++++ .../runners/dask/transform_evaluator.py | 18 +++++------------- 2 files changed, 12 insertions(+), 13 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index ecf7e920121f..aa55dc9beeb9 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -65,6 +65,13 @@ def mult_by(x, y): pcoll = p | beam.Create([1]) | beam.Map(mult_by, 3) assert_that(pcoll, equal_to([3])) + def test_map_with_named_side_inputs(self): + def mult_by(x, y): + return x * y + + with self.pipeline as p: + pcoll = p | beam.Create([1]) | beam.Map(mult_by, y=3) + assert_that(pcoll, equal_to([3])) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 3f4676c8f3e2..d10bdc94f0d1 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -43,13 +43,7 @@ class DaskBagOp(abc.ABC): @property def side_inputs(self): - return self.applied.side_inputs - - @functools.cached_property - def named_inputs(self): - named_inputs = self.applied.named_inputs() - del named_inputs['None'] - return named_inputs + return self.applied.transform.args @abc.abstractmethod def apply(self, input_bag: OpInput) -> db.Bag: @@ -71,16 +65,14 @@ def apply(self, input_bag: OpInput) -> db.Bag: class ParDo(DaskBagOp): def apply(self, input_bag: OpInput) -> db.Bag: - fn = t.cast(apache_beam.ParDo, self.applied.transform).fn - print(self.side_inputs) - print(self.named_inputs) - return input_bag.map(fn.process, *self.side_inputs, **self.named_inputs).flatten() + transform = t.cast(apache_beam.ParDo, self.applied.transform) + return input_bag.map(transform.fn.process, *transform.args, **transform.kwargs).flatten() class Map(DaskBagOp): def apply(self, input_bag: OpInput) -> db.Bag: - fn = t.cast(apache_beam.Map, self.applied.transform).fn - return input_bag.map(fn.process, *self.side_inputs, **self.named_inputs) + transform = t.cast(apache_beam.Map, self.applied.transform) + return input_bag.map(transform.fn.process, *transform.args, **transform.kwargs) class GroupByKey(DaskBagOp): From c62050e33b638e48f1b2e7989a902acc309909b8 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Mon, 3 Oct 2022 00:25:44 -0400 Subject: [PATCH 037/173] WIP Windowing with dask runner. --- .../apache_beam/runners/dask/dask_runner.py | 4 +- .../apache_beam/runners/dask/overrides.py | 5 +- .../runners/dask/transform_evaluator.py | 160 +++++++++++++++++- 3 files changed, 158 insertions(+), 11 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index 74915c4e254c..1b1ca65fefa1 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -25,7 +25,7 @@ import argparse import typing as t -from apache_beam import pvalue +from apache_beam import pvalue, TaggedOutput from apache_beam.options.pipeline_options import PipelineOptions from apache_beam.pipeline import AppliedPTransform from apache_beam.pipeline import PipelineVisitor @@ -35,7 +35,9 @@ from apache_beam.runners.direct.direct_runner import BundleBasedDirectRunner from apache_beam.runners.runner import PipelineResult from apache_beam.runners.runner import PipelineState +from apache_beam.transforms.window import WindowFn, TimestampedValue, GlobalWindow from apache_beam.utils.interactive_utils import is_in_notebook +from apache_beam.utils.windowed_value import WindowedValue class DaskOptions(PipelineOptions): diff --git a/sdks/python/apache_beam/runners/dask/overrides.py b/sdks/python/apache_beam/runners/dask/overrides.py index f06b6de6a10e..ab159c7c7ece 100644 --- a/sdks/python/apache_beam/runners/dask/overrides.py +++ b/sdks/python/apache_beam/runners/dask/overrides.py @@ -15,7 +15,6 @@ # limitations under the License. # import dataclasses - import typing as t import apache_beam as beam @@ -74,8 +73,6 @@ def infer_output_type(self, input_type): @typehints.with_input_types(t.Tuple[K, t.Iterable[V]]) @typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) class _GroupAlsoByWindow(beam.ParDo): - """Not used yet...""" - def __init__(self, windowing): super().__init__(_GroupAlsoByWindowDoFn(windowing)) self.windowing = windowing @@ -91,7 +88,9 @@ class _GroupByKey(beam.PTransform): def expand(self, input_or_inputs): return ( input_or_inputs + | "ReifyWindows" >> beam.ParDo(beam.GroupByKey.ReifyWindows()) | "GroupByKey" >> _GroupByKeyOnly() + | "GroupByWindow" >> _GroupAlsoByWindow(input_or_inputs.windowing) ) diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index d10bdc94f0d1..71cf4eb21e34 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -21,6 +21,7 @@ TODO(alxr): Translate ops from https://docs.dask.org/en/latest/bag-api.html. """ +import contextlib import dataclasses import abc @@ -29,21 +30,115 @@ import functools import apache_beam +from apache_beam import TaggedOutput, DoFn +from apache_beam.internal import util from apache_beam.pipeline import AppliedPTransform +from apache_beam.runners.common import DoFnContext, DoFnSignature, Receiver, _OutputHandler, DoFnInvoker from apache_beam.runners.dask.overrides import _Create from apache_beam.runners.dask.overrides import _Flatten from apache_beam.runners.dask.overrides import _GroupByKeyOnly +from apache_beam.transforms.sideinputs import SideInputMap +from apache_beam.transforms.window import WindowFn, TimestampedValue, GlobalWindow +from apache_beam.utils.windowed_value import WindowedValue OpInput = t.Union[db.Bag, t.Sequence[db.Bag], None] +@dataclasses.dataclass +class WindowAccessor: + window_fn: WindowFn + + def __getitem__(self, item: t.Any): + if isinstance(item, TaggedOutput): + item = item.value + + if isinstance(item, WindowedValue): + windowed_value = item + elif isinstance(item, TimestampedValue): + assign_context = WindowFn.AssignContext(item.timestamp, item.value) + windowed_value = WindowedValue(item.value, item.timestamp, + self.window_fn.assign(assign_context)) + else: + windowed_value = WindowedValue(item, 0, (GlobalWindow(),)) + + return windowed_value + + +@dataclasses.dataclass +class TaggingReceiver(Receiver): + tag: str + values: t.List[t.Union[WindowedValue, t.Any]] + + def receive(self, windowed_value: WindowedValue): + if self.tag: + output = TaggedOutput(self.tag, windowed_value) + else: + output = windowed_value + self.values.append(output) + + +@dataclasses.dataclass +class OneReceiver(dict): + values: t.List[t.Union[WindowedValue, t.Any]] + + def __missing__(self, key): + if key not in self: + self[key] = TaggingReceiver(key, self.values) + return self[key] + + +@dataclasses.dataclass +class DoFnWorker: + label: str + map_fn: DoFn + window_fn: WindowFn + side_inputs: t.List[SideInputMap] + args: t.Any + kwargs: t.Any + + def __post_init__(self): + self._values = [] + + tagged_receivers = OneReceiver(self._values) + do_fn_signature = DoFnSignature(self.map_fn) + output_handler = _OutputHandler( + window_fn=self.window_fn, + main_receivers=tagged_receivers[None], + tagged_receivers=tagged_receivers, + per_element_output_counter=None, + ) + + self._invoker = DoFnInvoker.create_invoker( + do_fn_signature, + output_handler, + DoFnContext(self.label, state=None), + self.side_inputs, + self.args, + self.kwargs, + user_state_context=None, + bundle_finalizer_param=DoFn.BundleFinalizerParam(), + ) + + def __del__(self): + self._invoker.invoke_teardown() + + def invoke(self, items): + try: + self._invoker.invoke_setup() + self._invoker.invoke_start_bundle() + + self._invoker.invoke_process() + + finally: + self._invoker.invoke_finish_bundle() + @dataclasses.dataclass class DaskBagOp(abc.ABC): applied: AppliedPTransform @property - def side_inputs(self): - return self.applied.transform.args + def transform(self): + return self.applied.transform @abc.abstractmethod def apply(self, input_bag: OpInput) -> db.Bag: @@ -58,21 +153,72 @@ def apply(self, input_bag: OpInput) -> db.Bag: class Create(DaskBagOp): def apply(self, input_bag: OpInput) -> db.Bag: assert input_bag is None, 'Create expects no input!' - original_transform = t.cast(_Create, self.applied.transform) + original_transform = t.cast(_Create, self.transform) items = original_transform.values return db.from_sequence(items) class ParDo(DaskBagOp): + def apply(self, input_bag: OpInput) -> db.Bag: - transform = t.cast(apache_beam.ParDo, self.applied.transform) - return input_bag.map(transform.fn.process, *transform.args, **transform.kwargs).flatten() + transform = t.cast(apache_beam.ParDo, self.transform) + + label = transform.label + map_fn = transform.fn + args, kwargs = transform.raw_side_inputs + main_input = next(iter(self.applied.main_inputs.values())) + window_fn = main_input.windowing.windowfn if hasattr(main_input, "windowing") else None + + context = DoFnContext(label, state=None) + bundle_finalizer_param = DoFn.BundleFinalizerParam() + do_fn_signature = DoFnSignature(map_fn) + + values = [] + + tagged_receivers = OneReceiver(values) + + output_processor = _OutputHandler( + window_fn=window_fn, + main_receivers=tagged_receivers[None], + tagged_receivers=tagged_receivers, + per_element_output_counter=None, + ) + + do_fn_invoker = DoFnInvoker.create_invoker( + do_fn_signature, + output_processor, + context, + None, + args, + kwargs, + user_state_context=None, + bundle_finalizer_param=bundle_finalizer_param) + + # Invoke setup just in case + do_fn_invoker.invoke_setup() + do_fn_invoker.invoke_start_bundle() + + for input_item in batch: + windowed_value = get_windowed_value(input_item, window_fn) + do_fn_invoker.invoke_process(windowed_value) + + do_fn_invoker.invoke_finish_bundle() + # Invoke teardown just in case + do_fn_invoker.invoke_teardown() + + # This has to happen last as we might receive results + # in invoke_finish_bundle() or invoke_teardown() + ret = list(values) + + return input_bag.map(transform.fn.process, *args, **kwargs).flatten() class Map(DaskBagOp): def apply(self, input_bag: OpInput) -> db.Bag: - transform = t.cast(apache_beam.Map, self.applied.transform) - return input_bag.map(transform.fn.process, *transform.args, **transform.kwargs) + transform = t.cast(apache_beam.Map, self.transform) + args, kwargs = util.insert_values_in_args( + transform.args, transform.kwargs, transform.side_inputs) + return input_bag.map(transform.fn.process, *args, **kwargs) class GroupByKey(DaskBagOp): From 79d4603f4d54a69b6f34b3d3c2769018798b5cfa Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Wed, 22 Jun 2022 16:26:02 -0700 Subject: [PATCH 038/173] WIP: Created a skeleton dask runner implementation. --- .../apache_beam/options/pipeline_options.py | 1 + .../apache_beam/runners/dask/dask_runner.py | 123 ++++++++++++++++++ 2 files changed, 124 insertions(+) create mode 100644 sdks/python/apache_beam/runners/dask/dask_runner.py diff --git a/sdks/python/apache_beam/options/pipeline_options.py b/sdks/python/apache_beam/options/pipeline_options.py index 7fa9396ed2e9..4d545372041c 100644 --- a/sdks/python/apache_beam/options/pipeline_options.py +++ b/sdks/python/apache_beam/options/pipeline_options.py @@ -455,6 +455,7 @@ class StandardOptions(PipelineOptions): 'apache_beam.runners.portability.spark_runner.SparkRunner', 'apache_beam.runners.test.TestDirectRunner', 'apache_beam.runners.test.TestDataflowRunner', + 'apache_beam.runners.dask.dask_runner.DaskRunner', ) KNOWN_RUNNER_NAMES = [path.split('.')[-1] for path in ALL_KNOWN_RUNNERS] diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py new file mode 100644 index 000000000000..b97426184e8c --- /dev/null +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -0,0 +1,123 @@ +# +# 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. +# + +"""DaskRunner, executing remote jobs on Dask.distributed. + +The DaskRunner is a runner implementation that executes a graph of +transformations across processes and workers via Dask distributed's +scheduler. +""" +import typing as t +import argparse +import dataclasses + +from apache_beam import pvalue +from apache_beam.options.pipeline_options import PipelineOptions +from apache_beam.pvalue import PValue +from apache_beam.runners.direct.consumer_tracking_pipeline_visitor import ConsumerTrackingPipelineVisitor +from apache_beam.utils.interactive_utils import is_in_notebook + +from apache_beam.runners.direct.direct_runner import BundleBasedDirectRunner + +from apache_beam.pipeline import PipelineVisitor, AppliedPTransform + +import dask.bag as db + + +class DaskOptions(PipelineOptions): + + @classmethod + def _add_argparse_args(cls, parser: argparse.ArgumentParser) -> None: + # TODO: get Dask client options + pass + + +@dataclasses.dataclass +class DaskExecutor: + value_to_consumers: t.Dict[pvalue.PValue, t.Set[AppliedPTransform]] + # root_transforms: t.Set[AppliedPTransform] + step_names: t.Dict[AppliedPTransform, str] + views: t.List[pvalue.AsSideInput] + + def __post_init__(self): + # TODO(alxr): Translate to Bags + + pass + + def start(self, roots: t.Set[AppliedPTransform]) -> None: + pass + + def await_completion(self) -> None: + pass + + def shutdown(self) -> None: + pass + + +class DaskRunner(BundleBasedDirectRunner): + """Executes a pipeline on a Dask distributed client.""" + + @staticmethod + def to_dask_bag_visitor(self): + + @dataclasses.dataclass + class DaskBagVisitor(PipelineVisitor): + + def visit_transform(self, transform_node: AppliedPTransform) -> None: + inputs = list(transform_node.inputs) + pass + + + + @staticmethod + def is_fnapi_compatible(): + return False + + def run_pipeline(self, pipeline, options): + # TODO(alxr): Create interactive notebook support. + if is_in_notebook(): + raise NotImplementedError('interactive support will come later!') + + try: + import dask.bag as db + import dask.distributed as ddist + except ImportError: + raise ImportError('DaskRunner is not available. Please install apache_beam[dask].') + + dask_options = options.view_as(DaskOptions) + + self.client = ddist.Client(**dask_options.get_all_options()) + + self.consumer_tracking_visitor = ConsumerTrackingPipelineVisitor() + pipeline.visit(self.consumer_tracking_visitor) + + dask_visitor = self.to_dask_bag_visitor() + pipeline.visit(dask_visitor) + + + # if pipeline: + # pass + # else: + # raise ValueError('Proto or FunctionAPI environments are not supported.') + # if pipeline: + # + # # Flatten / Optimize graph? + # + # # Trigger a traversal of all reachable nodes. + # self.visit_transforms(pipeline, options) + # + # Get API Client? \ No newline at end of file From 248ec70708b43fedb7d9a40ae144af3249fadf25 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Thu, 23 Jun 2022 11:08:49 -0700 Subject: [PATCH 039/173] WIP: Idea for a translation evaluator. --- .../apache_beam/runners/dask/dask_runner.py | 56 +++++++++++++++---- .../runners/dask/transform_evaluator.py | 52 +++++++++++++++++ 2 files changed, 98 insertions(+), 10 deletions(-) create mode 100644 sdks/python/apache_beam/runners/dask/transform_evaluator.py diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index b97426184e8c..579d874a0df2 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -21,6 +21,8 @@ transformations across processes and workers via Dask distributed's scheduler. """ +import functools +import itertools import typing as t import argparse import dataclasses @@ -49,17 +51,27 @@ def _add_argparse_args(cls, parser: argparse.ArgumentParser) -> None: @dataclasses.dataclass class DaskExecutor: value_to_consumers: t.Dict[pvalue.PValue, t.Set[AppliedPTransform]] - # root_transforms: t.Set[AppliedPTransform] + root_transforms: t.Set[AppliedPTransform] step_names: t.Dict[AppliedPTransform, str] views: t.List[pvalue.AsSideInput] + _root_nodes = None + _all_nodes = None - def __post_init__(self): - # TODO(alxr): Translate to Bags + @property + @functools.cached_property + def root_nodes(self): + return frozenset(self.root_transforms) + + @property + @functools.cached_property + def all_nodes(self): + return itertools.chain( + self.root_nodes, *itertools.chain(self.value_to_consumers.values()))) + + def start(self) -> None: + for root in self.root_nodes: - pass - def start(self, roots: t.Set[AppliedPTransform]) -> None: - pass def await_completion(self) -> None: pass @@ -72,14 +84,38 @@ class DaskRunner(BundleBasedDirectRunner): """Executes a pipeline on a Dask distributed client.""" @staticmethod - def to_dask_bag_visitor(self): + def to_dask_bag_visitor(self) -> PipelineVisitor: @dataclasses.dataclass class DaskBagVisitor(PipelineVisitor): - def visit_transform(self, transform_node: AppliedPTransform) -> None: - inputs = list(transform_node.inputs) - pass + value_to_consumers = { + } # type: Dict[pvalue.PValue, Set[AppliedPTransform]] + root_transforms = set() # type: Set[AppliedPTransform] + step_names = {} # type: Dict[AppliedPTransform, str] + + def __post_init__(self): + self._num_transforms = 0 + self._views = set() + + def visit_transform(self, applied_ptransform: AppliedPTransform) -> None: + inputs = list(applied_ptransform.inputs) + if inputs: + for input_value in inputs: + if isinstance(input_value, pvalue.PBegin): + self.root_transforms.add(applied_ptransform) + if input_value not in self.value_to_consumers: + self.value_to_consumers[input_value] = set() + self.value_to_consumers[input_value].add(applied_ptransform) + else: + self.root_transforms.add(applied_ptransform) + self.step_names[applied_ptransform] = 's%d' % (self._num_transforms) + self._num_transforms += 1 + + for side_input in applied_ptransform.side_inputs: + self._views.add(side_input) + + return DaskBagVisitor() diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py new file mode 100644 index 000000000000..cc67bce5befe --- /dev/null +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -0,0 +1,52 @@ +import typing as t +import abc +import dataclasses + +import apache_beam +from apache_beam.pipeline import AppliedPTransform + +import dask.bag as db + + +@dataclasses.dataclass +class DaskOp(abc.ABC): + application: AppliedPTransform + side_inputs: t.Sequence[t.Any] + + @abc.abstractmethod + def apply(self, element): + pass + + +class NoOp(DaskOp): + def apply(self, element): + return element + + +class Create(DaskOp): + def apply(self, element): + assert element is None, 'Create expects no input!' + + original_transform = t.cast(apache_beam.Create, self.application.transform) + items = original_transform.values + return db.from_sequence(items) + + +class Impulse(DaskOp): + def apply(self, element): + raise NotImplementedError() + + +class ParDo(DaskOp): + def apply(self, element): + assert element is not None, 'ParDo must receive input!' + assert isinstance(element, db.Bag) + assert self.application is not None + transform = self.application.transform + assert isinstance(transform, apache_beam.ParDo) + + +TRANSLATIONS = { + apache_beam.Create: Create + +} \ No newline at end of file From 42452ca3873d2af88d687d1187760d4e7e533bb1 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 2 Jul 2022 00:35:38 -0700 Subject: [PATCH 040/173] Added overrides and a visitor that translates operations. --- .../apache_beam/runners/dask/dask_runner.py | 110 +++++--------- .../runners/dask/dask_runner_test.py | 50 +++++++ .../apache_beam/runners/dask/overrides.py | 139 ++++++++++++++++++ .../runners/dask/transform_evaluator.py | 50 +++++-- 4 files changed, 264 insertions(+), 85 deletions(-) create mode 100644 sdks/python/apache_beam/runners/dask/dask_runner_test.py create mode 100644 sdks/python/apache_beam/runners/dask/overrides.py diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index 579d874a0df2..daee5e6b65b5 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -20,24 +20,32 @@ The DaskRunner is a runner implementation that executes a graph of transformations across processes and workers via Dask distributed's scheduler. + + +Ideas to explore / Notes: +- Write a PCollection subclass that wraps a Dask Bag. + - Would be the input + return of the translation operators. +- The Ray runner is more focused on Task scheduling; This should focus more on graph translation. + +- A bundle is a subset of elements in a PCollection. i.e. a small set of elements that are processed together. +- In Dask, it is probably the same as a partition. Thus, we probably don't need to worry about it; Dask should take + care of it. """ -import functools -import itertools -import typing as t import argparse import dataclasses +import typing as t + +from dask import bag as db from apache_beam import pvalue from apache_beam.options.pipeline_options import PipelineOptions -from apache_beam.pvalue import PValue -from apache_beam.runners.direct.consumer_tracking_pipeline_visitor import ConsumerTrackingPipelineVisitor -from apache_beam.utils.interactive_utils import is_in_notebook - -from apache_beam.runners.direct.direct_runner import BundleBasedDirectRunner - from apache_beam.pipeline import PipelineVisitor, AppliedPTransform - -import dask.bag as db +from apache_beam.pvalue import PCollection +from apache_beam.runners.dask.overrides import dask_overrides +from apache_beam.runners.dask.transform_evaluator import TRANSLATIONS, NoOp +# from apache_beam.runners.direct.consumer_tracking_pipeline_visitor import ConsumerTrackingPipelineVisitor +from apache_beam.runners.direct.direct_runner import BundleBasedDirectRunner +from apache_beam.utils.interactive_utils import is_in_notebook class DaskOptions(PipelineOptions): @@ -48,77 +56,34 @@ def _add_argparse_args(cls, parser: argparse.ArgumentParser) -> None: pass -@dataclasses.dataclass -class DaskExecutor: - value_to_consumers: t.Dict[pvalue.PValue, t.Set[AppliedPTransform]] - root_transforms: t.Set[AppliedPTransform] - step_names: t.Dict[AppliedPTransform, str] - views: t.List[pvalue.AsSideInput] - _root_nodes = None - _all_nodes = None - - @property - @functools.cached_property - def root_nodes(self): - return frozenset(self.root_transforms) - - @property - @functools.cached_property - def all_nodes(self): - return itertools.chain( - self.root_nodes, *itertools.chain(self.value_to_consumers.values()))) - - def start(self) -> None: - for root in self.root_nodes: - - - - def await_completion(self) -> None: - pass - - def shutdown(self) -> None: - pass - - class DaskRunner(BundleBasedDirectRunner): """Executes a pipeline on a Dask distributed client.""" @staticmethod - def to_dask_bag_visitor(self) -> PipelineVisitor: + def to_dask_bag_visitor() -> PipelineVisitor: @dataclasses.dataclass class DaskBagVisitor(PipelineVisitor): + bags: t.Dict[AppliedPTransform, db.Bag] = dataclasses.fields(default_factory=dict) - value_to_consumers = { - } # type: Dict[pvalue.PValue, Set[AppliedPTransform]] - root_transforms = set() # type: Set[AppliedPTransform] - step_names = {} # type: Dict[AppliedPTransform, str] + def visit_transform(self, transform_node: AppliedPTransform) -> None: + op_class = TRANSLATIONS.get(transform_node.transform.__class__, NoOp) + op = op_class(transform_node) - def __post_init__(self): - self._num_transforms = 0 - self._views = set() - - def visit_transform(self, applied_ptransform: AppliedPTransform) -> None: - inputs = list(applied_ptransform.inputs) + inputs = list(transform_node.inputs) if inputs: for input_value in inputs: if isinstance(input_value, pvalue.PBegin): - self.root_transforms.add(applied_ptransform) - if input_value not in self.value_to_consumers: - self.value_to_consumers[input_value] = set() - self.value_to_consumers[input_value].add(applied_ptransform) - else: - self.root_transforms.add(applied_ptransform) - self.step_names[applied_ptransform] = 's%d' % (self._num_transforms) - self._num_transforms += 1 + self.bags[transform_node] = op.apply(None) - for side_input in applied_ptransform.side_inputs: - self._views.add(side_input) + prev_op = input_value.producer + if prev_op in self.bags: + self.bags[transform_node] = op.apply(self.bags[prev_op]) + else: + self.bags[transform_node] = op.apply(None) return DaskBagVisitor() - - @staticmethod def is_fnapi_compatible(): return False @@ -134,16 +99,19 @@ def run_pipeline(self, pipeline, options): except ImportError: raise ImportError('DaskRunner is not available. Please install apache_beam[dask].') - dask_options = options.view_as(DaskOptions) + # TODO(alxr): Wire up a real dask client + # dask_options = options.view_as(DaskOptions).get_all_options() + # self.client = ddist.Client(**dask_options) - self.client = ddist.Client(**dask_options.get_all_options()) + pipeline.replace_all(dask_overrides()) - self.consumer_tracking_visitor = ConsumerTrackingPipelineVisitor() - pipeline.visit(self.consumer_tracking_visitor) + # consumer_tracking_visitor = ConsumerTrackingPipelineVisitor() + # pipeline.visit(consumer_tracking_visitor) dask_visitor = self.to_dask_bag_visitor() pipeline.visit(dask_visitor) + print(dask_visitor) # if pipeline: # pass @@ -156,4 +124,4 @@ def run_pipeline(self, pipeline, options): # # Trigger a traversal of all reachable nodes. # self.visit_transforms(pipeline, options) # - # Get API Client? \ No newline at end of file + # Get API Client? diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py new file mode 100644 index 000000000000..377e3b37bbaf --- /dev/null +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -0,0 +1,50 @@ +# +# 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.runners.dask.dask_runner import DaskRunner +from apache_beam.testing import test_pipeline + + +class DaskRunnerRunPipelineTest(unittest.TestCase): + """Test class used to introspect the dask runner via a debugger.""" + + def test_create(self): + p = test_pipeline.TestPipeline(runner=DaskRunner()) + _ = p | beam.Create([1]) + p.run() + + def test_create_and_map(self): + def double(x): + return x * 2 + + p = test_pipeline.TestPipeline(runner=DaskRunner()) + _ = p | beam.Create([1]) | beam.Map(double) + p.run() + + def test_create_map_and_groupby(self): + def double(x): + return x * 2, x + + p = test_pipeline.TestPipeline(runner=DaskRunner()) + _ = p | beam.Create([1]) | beam.Map(double) | beam.GroupByKey() + p.run() + + +if __name__ == '__main__': + unittest.main() diff --git a/sdks/python/apache_beam/runners/dask/overrides.py b/sdks/python/apache_beam/runners/dask/overrides.py new file mode 100644 index 000000000000..38f4701ed76a --- /dev/null +++ b/sdks/python/apache_beam/runners/dask/overrides.py @@ -0,0 +1,139 @@ +# +# 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 dataclasses +import typing as t + +import apache_beam as beam +from apache_beam import ( + Create, + GroupByKey, + ParDo, + PTransform, + pvalue, + Windowing, +) +from apache_beam import typehints +from apache_beam.io.iobase import SourceBase +from apache_beam.pipeline import PTransformOverride, AppliedPTransform +from apache_beam.runners.direct.direct_runner import _GroupAlsoByWindowDoFn +from apache_beam.transforms import ptransform +from apache_beam.transforms.window import GlobalWindows + +K = t.TypeVar("K") +V = t.TypeVar("V") + + +@dataclasses.dataclass +class _Create(PTransform): + values: t.Tuple[t.Any] + + def expand(self, input_or_inputs): + return pvalue.PCollection.from_(input_or_inputs) + + def get_windowing(self, inputs): + # type: (typing.Any) -> Windowing + return Windowing(GlobalWindows()) + + +@typehints.with_input_types(K) +@typehints.with_output_types(K) +class _Reshuffle(PTransform): + def expand(self, input_or_inputs): + return pvalue.PCollection.from_(input_or_inputs) + + +@dataclasses.dataclass +class _Read(PTransform): + source: SourceBase + + def expand(self, input_or_inputs): + return pvalue.PCollection.from_(input_or_inputs) + + +@typehints.with_input_types(t.Tuple[K, V]) +@typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) +class _GroupByKeyOnly(PTransform): + def expand(self, input_or_inputs): + return pvalue.PCollection.from_(input_or_inputs) + + def infer_output_type(self, input_type): + key_type, value_type = typehints.trivial_inference.key_value_types(input_type) + return typehints.KV[key_type, typehints.Iterable[value_type]] + + +@typehints.with_input_types(t.Tuple[K, t.Iterable[V]]) +@typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) +class _GroupAlsoByWindow(ParDo): + def __init__(self, windowing): + super(_GroupAlsoByWindow, self).__init__( + _GroupAlsoByWindowDoFn(windowing)) + self.windowing = windowing + + def expand(self, input_or_inputs): + return pvalue.PCollection.from_(input_or_inputs) + + +@typehints.with_input_types(t.Tuple[K, V]) +@typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) +class _GroupByKey(PTransform): + def expand(self, input_or_inputs): + return ( + input_or_inputs + | "ReifyWindows" >> ParDo(GroupByKey.ReifyWindows()) + | "GroupByKey" >> _GroupByKeyOnly() + | "GroupByWindow" >> _GroupAlsoByWindow(input_or_inputs.windowing)) + + +def dask_overrides() -> t.List[PTransformOverride]: + class CreateOverride(PTransformOverride): + def matches(self, applied_ptransform: AppliedPTransform) -> bool: + return applied_ptransform.transform.__class__ == beam.Create + + def get_replacement_transform_for_applied_ptransform( + self, applied_ptransform: AppliedPTransform) -> ptransform.PTransform: + return _Create(t.cast(Create, applied_ptransform.transform).values) + + class ReshuffleOverride(PTransformOverride): + def matches(self, applied_ptransform: AppliedPTransform) -> bool: + return applied_ptransform.transform.__class__ == beam.Reshuffle + + def get_replacement_transform_for_applied_ptransform( + self, applied_ptransform: AppliedPTransform) -> ptransform.PTransform: + return _Reshuffle() + + class ReadOverride(PTransformOverride): + def matches(self, applied_ptransform: AppliedPTransform) -> bool: + return applied_ptransform.transform.__class__ == beam.io.Read + + def get_replacement_transform_for_applied_ptransform( + self, applied_ptransform: AppliedPTransform) -> ptransform.PTransform: + return _Read(t.cast(beam.io.Read, applied_ptransform.transform).source) + + class GroupByKeyOverride(PTransformOverride): + def matches(self, applied_ptransform: AppliedPTransform) -> bool: + return applied_ptransform.transform.__class__ == beam.GroupByKey + + def get_replacement_transform_for_applied_ptransform( + self, applied_ptransform: AppliedPTransform) -> ptransform.PTransform: + return _GroupByKey() + + return [ + CreateOverride(), + ReshuffleOverride(), + ReadOverride(), + GroupByKeyOverride(), + ] diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index cc67bce5befe..fbcf498eb051 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -1,3 +1,19 @@ +# +# 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 typing as t import abc import dataclasses @@ -7,24 +23,29 @@ import dask.bag as db +from apache_beam.runners.dask.overrides import _Create + @dataclasses.dataclass -class DaskOp(abc.ABC): +class DaskBagOp(abc.ABC): application: AppliedPTransform - side_inputs: t.Sequence[t.Any] + + @property + def side_inputs(self): + return self.application.side_inputs @abc.abstractmethod - def apply(self, element): + def apply(self, element: db.Bag) -> db.Bag: pass -class NoOp(DaskOp): - def apply(self, element): +class NoOp(DaskBagOp): + def apply(self, element: db.Bag) -> db.Bag: return element -class Create(DaskOp): - def apply(self, element): +class Create(DaskBagOp): + def apply(self, element: db.Bag) -> db.Bag: assert element is None, 'Create expects no input!' original_transform = t.cast(apache_beam.Create, self.application.transform) @@ -32,21 +53,22 @@ def apply(self, element): return db.from_sequence(items) -class Impulse(DaskOp): - def apply(self, element): +class Impulse(DaskBagOp): + def apply(self, element: db.Bag) -> db.Bag: raise NotImplementedError() -class ParDo(DaskOp): - def apply(self, element): +class ParDo(DaskBagOp): + def apply(self, element: db.Bag) -> db.Bag: assert element is not None, 'ParDo must receive input!' assert isinstance(element, db.Bag) assert self.application is not None transform = self.application.transform assert isinstance(transform, apache_beam.ParDo) + return element -TRANSLATIONS = { - apache_beam.Create: Create -} \ No newline at end of file +TRANSLATIONS = { + _Create: Create +} From 1da2ddd90126e452f1f6b093e5d7aadecc1dafb4 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 2 Jul 2022 00:36:43 -0700 Subject: [PATCH 041/173] Fixed a dataclass typo. --- sdks/python/apache_beam/runners/dask/dask_runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index daee5e6b65b5..0693a98bbec3 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -64,7 +64,7 @@ def to_dask_bag_visitor() -> PipelineVisitor: @dataclasses.dataclass class DaskBagVisitor(PipelineVisitor): - bags: t.Dict[AppliedPTransform, db.Bag] = dataclasses.fields(default_factory=dict) + bags: t.Dict[AppliedPTransform, db.Bag] = dataclasses.field(default_factory=dict) def visit_transform(self, transform_node: AppliedPTransform) -> None: op_class = TRANSLATIONS.get(transform_node.transform.__class__, NoOp) From 14885a3b1452a31c152eb91800a222d8634727be Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 2 Jul 2022 01:06:54 -0700 Subject: [PATCH 042/173] Expanded translations. --- .../runners/dask/transform_evaluator.py | 54 +++++++++++-------- 1 file changed, 32 insertions(+), 22 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index fbcf498eb051..3f500e0d8e18 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -14,6 +14,10 @@ # See the License for the specific language governing permissions and # limitations under the License. # +""" + +TODO(alxr): Translate ops from https://docs.dask.org/en/latest/bag-api.html. +""" import typing as t import abc import dataclasses @@ -23,52 +27,58 @@ import dask.bag as db -from apache_beam.runners.dask.overrides import _Create +from apache_beam.runners.dask.overrides import _Create, _GroupByKeyOnly @dataclasses.dataclass class DaskBagOp(abc.ABC): - application: AppliedPTransform + applied: AppliedPTransform @property def side_inputs(self): - return self.application.side_inputs + return self.applied.side_inputs @abc.abstractmethod - def apply(self, element: db.Bag) -> db.Bag: + def apply(self, input_bag: t.Optional[db.Bag]) -> db.Bag: pass class NoOp(DaskBagOp): - def apply(self, element: db.Bag) -> db.Bag: - return element + def apply(self, input_bag: t.Optional[db.Bag]) -> db.Bag: + return input_bag class Create(DaskBagOp): - def apply(self, element: db.Bag) -> db.Bag: - assert element is None, 'Create expects no input!' - - original_transform = t.cast(apache_beam.Create, self.application.transform) + def apply(self, input_bag: t.Optional[db.Bag]) -> db.Bag: + assert input_bag is None, 'Create expects no input!' + original_transform = t.cast(_Create, self.applied.transform) items = original_transform.values return db.from_sequence(items) -class Impulse(DaskBagOp): - def apply(self, element: db.Bag) -> db.Bag: - raise NotImplementedError() +class ParDo(DaskBagOp): + def apply(self, input_bag: t.Optional[db.Bag]) -> db.Bag: + fn = t.cast(apache_beam.ParDo, self.applied.transform).fn + return input_bag.map(fn).flatten() + +class Map(DaskBagOp): + def apply(self, input_bag: t.Optional[db.Bag]) -> db.Bag: + fn = t.cast(apache_beam.Map, self.applied.transform).fn + return input_bag.map(fn) -class ParDo(DaskBagOp): - def apply(self, element: db.Bag) -> db.Bag: - assert element is not None, 'ParDo must receive input!' - assert isinstance(element, db.Bag) - assert self.application is not None - transform = self.application.transform - assert isinstance(transform, apache_beam.ParDo) - return element +class GroupByKey(DaskBagOp): + def apply(self, input_bag: t.Optional[db.Bag]) -> db.Bag: + def key(item): + return item[0] + + return input_bag.groupby(key) TRANSLATIONS = { - _Create: Create + _Create: Create, + apache_beam.ParDo: ParDo, + apache_beam.Map: Map, + _GroupByKeyOnly: GroupByKey, } From fca24200fe0e76adc7baba946a011998b3b1b07d Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 2 Jul 2022 01:33:30 -0700 Subject: [PATCH 043/173] Core idea seems to be kinda working... --- .../apache_beam/runners/dask/__init__.py | 0 .../apache_beam/runners/dask/dask_runner.py | 22 +++++++++---------- .../apache_beam/runners/dask/overrides.py | 4 +++- .../runners/dask/transform_evaluator.py | 4 ++-- 4 files changed, 15 insertions(+), 15 deletions(-) create mode 100644 sdks/python/apache_beam/runners/dask/__init__.py diff --git a/sdks/python/apache_beam/runners/dask/__init__.py b/sdks/python/apache_beam/runners/dask/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index 0693a98bbec3..1b4910a25af9 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -35,15 +35,11 @@ import dataclasses import typing as t -from dask import bag as db - from apache_beam import pvalue from apache_beam.options.pipeline_options import PipelineOptions from apache_beam.pipeline import PipelineVisitor, AppliedPTransform -from apache_beam.pvalue import PCollection from apache_beam.runners.dask.overrides import dask_overrides from apache_beam.runners.dask.transform_evaluator import TRANSLATIONS, NoOp -# from apache_beam.runners.direct.consumer_tracking_pipeline_visitor import ConsumerTrackingPipelineVisitor from apache_beam.runners.direct.direct_runner import BundleBasedDirectRunner from apache_beam.utils.interactive_utils import is_in_notebook @@ -60,7 +56,9 @@ class DaskRunner(BundleBasedDirectRunner): """Executes a pipeline on a Dask distributed client.""" @staticmethod - def to_dask_bag_visitor() -> PipelineVisitor: + def to_dask_bag_visitor() -> 'DaskBagVisitor': + + from dask import bag as db @dataclasses.dataclass class DaskBagVisitor(PipelineVisitor): @@ -99,20 +97,20 @@ def run_pipeline(self, pipeline, options): except ImportError: raise ImportError('DaskRunner is not available. Please install apache_beam[dask].') - # TODO(alxr): Wire up a real dask client - # dask_options = options.view_as(DaskOptions).get_all_options() - # self.client = ddist.Client(**dask_options) + # TODO(alxr): Actually use this right. + dask_options = options.view_as(DaskOptions).get_all_options(drop_default=True) + client = ddist.Client(**dask_options) pipeline.replace_all(dask_overrides()) - # consumer_tracking_visitor = ConsumerTrackingPipelineVisitor() - # pipeline.visit(consumer_tracking_visitor) - dask_visitor = self.to_dask_bag_visitor() pipeline.visit(dask_visitor) - print(dask_visitor) + for bag in dask_visitor.bags.values(): + bag.compute() + # TODO(alxr): Return the proper thing... + return None # if pipeline: # pass # else: diff --git a/sdks/python/apache_beam/runners/dask/overrides.py b/sdks/python/apache_beam/runners/dask/overrides.py index 38f4701ed76a..30df036eb7d9 100644 --- a/sdks/python/apache_beam/runners/dask/overrides.py +++ b/sdks/python/apache_beam/runners/dask/overrides.py @@ -78,6 +78,7 @@ def infer_output_type(self, input_type): @typehints.with_input_types(t.Tuple[K, t.Iterable[V]]) @typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) class _GroupAlsoByWindow(ParDo): + """Not used yet...""" def __init__(self, windowing): super(_GroupAlsoByWindow, self).__init__( _GroupAlsoByWindowDoFn(windowing)) @@ -90,6 +91,7 @@ def expand(self, input_or_inputs): @typehints.with_input_types(t.Tuple[K, V]) @typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) class _GroupByKey(PTransform): + """Not used yet...""" def expand(self, input_or_inputs): return ( input_or_inputs @@ -129,7 +131,7 @@ def matches(self, applied_ptransform: AppliedPTransform) -> bool: def get_replacement_transform_for_applied_ptransform( self, applied_ptransform: AppliedPTransform) -> ptransform.PTransform: - return _GroupByKey() + return _GroupByKeyOnly() return [ CreateOverride(), diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 3f500e0d8e18..e5b6ae00aba2 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -59,13 +59,13 @@ def apply(self, input_bag: t.Optional[db.Bag]) -> db.Bag: class ParDo(DaskBagOp): def apply(self, input_bag: t.Optional[db.Bag]) -> db.Bag: fn = t.cast(apache_beam.ParDo, self.applied.transform).fn - return input_bag.map(fn).flatten() + return input_bag.map(fn.process).flatten() class Map(DaskBagOp): def apply(self, input_bag: t.Optional[db.Bag]) -> db.Bag: fn = t.cast(apache_beam.Map, self.applied.transform).fn - return input_bag.map(fn) + return input_bag.map(fn.process) class GroupByKey(DaskBagOp): From 6dd1ada87c15dfb862032bbb2e34493c46e3ae1b Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sun, 3 Jul 2022 15:02:10 -0700 Subject: [PATCH 044/173] First iteration on DaskRunnerResult (keep track of pipeline state). --- .../apache_beam/runners/dask/dask_runner.py | 39 ++++++++++++++++--- .../runners/dask/dask_runner_test.py | 18 ++++----- 2 files changed, 42 insertions(+), 15 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index 1b4910a25af9..d70f4c3f5638 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -41,6 +41,7 @@ from apache_beam.runners.dask.overrides import dask_overrides from apache_beam.runners.dask.transform_evaluator import TRANSLATIONS, NoOp from apache_beam.runners.direct.direct_runner import BundleBasedDirectRunner +from apache_beam.runners.runner import PipelineResult, PipelineState from apache_beam.utils.interactive_utils import is_in_notebook @@ -48,16 +49,43 @@ class DaskOptions(PipelineOptions): @classmethod def _add_argparse_args(cls, parser: argparse.ArgumentParser) -> None: - # TODO: get Dask client options + # TODO(alxr): get Dask client options pass +@dataclasses.dataclass +class DaskRunnerResult(PipelineResult): + client: 'dask.distributed.Client' + futures: t.Sequence['dask.distributed.Future'] + + def __post_init__(self): + super().__init__(PipelineState.RUNNING) + + def wait_until_finish(self, duration=None) -> PipelineState: + try: + self.client.wait_for_workers(timeout=(duration / 1000)) + self._state = PipelineState.DONE + except: # pylint: disable=broad-except + self._state = PipelineState.FAILED + raise + return self._state + + def cancel(self) -> PipelineState: + self._state = PipelineState.CANCELLING + self.client.cancel(self.futures) + self._state = PipelineState.CANCELLED + return self._state + + def metrics(self): + # TODO(alxr): Collect and return metrics... + raise NotImplementedError('collecting metrics will come later!') + + class DaskRunner(BundleBasedDirectRunner): """Executes a pipeline on a Dask distributed client.""" @staticmethod def to_dask_bag_visitor() -> 'DaskBagVisitor': - from dask import bag as db @dataclasses.dataclass @@ -106,11 +134,10 @@ def run_pipeline(self, pipeline, options): dask_visitor = self.to_dask_bag_visitor() pipeline.visit(dask_visitor) - for bag in dask_visitor.bags.values(): - bag.compute() + futures = client.compute(list(dask_visitor.bags.values())) + + return DaskRunnerResult(client, futures) - # TODO(alxr): Return the proper thing... - return None # if pipeline: # pass # else: diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index 377e3b37bbaf..eab347bf5e7f 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -24,26 +24,26 @@ class DaskRunnerRunPipelineTest(unittest.TestCase): """Test class used to introspect the dask runner via a debugger.""" + def setUp(self) -> None: + self.p = test_pipeline.TestPipeline(runner=DaskRunner()) + def test_create(self): - p = test_pipeline.TestPipeline(runner=DaskRunner()) - _ = p | beam.Create([1]) - p.run() + _ = self.p | beam.Create([1]) + self.p.run() def test_create_and_map(self): def double(x): return x * 2 - p = test_pipeline.TestPipeline(runner=DaskRunner()) - _ = p | beam.Create([1]) | beam.Map(double) - p.run() + _ = self.p | beam.Create([1]) | beam.Map(double) + self.p.run() def test_create_map_and_groupby(self): def double(x): return x * 2, x - p = test_pipeline.TestPipeline(runner=DaskRunner()) - _ = p | beam.Create([1]) | beam.Map(double) | beam.GroupByKey() - p.run() + _ = self.p | beam.Create([1]) | beam.Map(double) | beam.GroupByKey() + self.p.run() if __name__ == '__main__': From 6675687961dd9c2ae658e730ed76ae6d3bac878a Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sun, 3 Jul 2022 21:16:24 -0700 Subject: [PATCH 045/173] Added minimal set of DaskRunner options. --- .../apache_beam/runners/dask/dask_runner.py | 30 ++++++++++++++++--- 1 file changed, 26 insertions(+), 4 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index d70f4c3f5638..cc8148e9edd4 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -14,7 +14,6 @@ # See the License for the specific language governing permissions and # limitations under the License. # - """DaskRunner, executing remote jobs on Dask.distributed. The DaskRunner is a runner implementation that executes a graph of @@ -35,6 +34,8 @@ import dataclasses import typing as t +import dask.distributed + from apache_beam import pvalue from apache_beam.options.pipeline_options import PipelineOptions from apache_beam.pipeline import PipelineVisitor, AppliedPTransform @@ -47,10 +48,28 @@ class DaskOptions(PipelineOptions): + @staticmethod + def _parse_timeout(candidate): + try: + return int(candidate) + except: # noqa + import dask + return dask.config.no_default + @classmethod def _add_argparse_args(cls, parser: argparse.ArgumentParser) -> None: - # TODO(alxr): get Dask client options - pass + parser.add_argument('--dask_client_address', type=str, default=None, + help='Address of a dask Scheduler server. Will default to a `dask.LocalCluster()`.') + parser.add_argument('--dask_connection_timeout', dest='timeout', type=DaskOptions._parse_timeout, + help='Timeout duration for initial connection to the scheduler.') + parser.add_argument('--dask_scheduler_file', type=str, default=None, + help='Path to a file with scheduler information if available.') + # TODO(alxr): Add options for security. + parser.add_argument('--dask_client_name', dest='name', type=str, default=None, + help='Gives the client a name that will be included in logs generated on the scheduler ' + 'for matters relating to this client.') + parser.add_argument('--dask_connection_limit', dest='connection_limit', type=int, default=512, + help='The number of open comms to maintain at once in the connection pool.') @dataclasses.dataclass @@ -63,7 +82,10 @@ def __post_init__(self): def wait_until_finish(self, duration=None) -> PipelineState: try: - self.client.wait_for_workers(timeout=(duration / 1000)) + if duration is not None: + # Convert milliseconds to seconds + duration /= 1000 + self.client.wait_for_workers(timeout=duration) self._state = PipelineState.DONE except: # pylint: disable=broad-except self._state = PipelineState.FAILED From 88ed36b7496085cc08dd77b03ba04d8afb3db92f Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Fri, 8 Jul 2022 15:15:08 -0700 Subject: [PATCH 046/173] WIP: Alllmost got asserts to work! The current status is: - CoGroupByKey is broken due to how tags are used with GroupByKey - GroupByKey should output `[('0', None), ('1', 1)]`, however it actually outputs: [(None, ('1', 1)), (None, ('0', None))] - Once that is fixed, we may have test pipelines work on Dask. --- .../apache_beam/runners/dask/dask_runner.py | 12 ++++++-- .../runners/dask/dask_runner_test.py | 7 +++-- .../apache_beam/runners/dask/overrides.py | 30 +++++++++++++++---- .../runners/dask/transform_evaluator.py | 20 +++++++++---- 4 files changed, 55 insertions(+), 14 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index cc8148e9edd4..4caab2cd475f 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -86,6 +86,7 @@ def wait_until_finish(self, duration=None) -> PipelineState: # Convert milliseconds to seconds duration /= 1000 self.client.wait_for_workers(timeout=duration) + self.client.gather(self.futures, errors='raise') self._state = PipelineState.DONE except: # pylint: disable=broad-except self._state = PipelineState.FAILED @@ -120,13 +121,20 @@ def visit_transform(self, transform_node: AppliedPTransform) -> None: inputs = list(transform_node.inputs) if inputs: + bag_inputs = [] for input_value in inputs: if isinstance(input_value, pvalue.PBegin): - self.bags[transform_node] = op.apply(None) + bag_inputs.append(None) prev_op = input_value.producer if prev_op in self.bags: - self.bags[transform_node] = op.apply(self.bags[prev_op]) + bag_inputs.append(self.bags[prev_op]) + + if len(bag_inputs) == 1: + self.bags[transform_node] = op.apply(bag_inputs[0]) + else: + self.bags[transform_node] = op.apply(bag_inputs) + else: self.bags[transform_node] = op.apply(None) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index eab347bf5e7f..a3788348336d 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -17,8 +17,10 @@ import unittest import apache_beam as beam +from apache_beam.runners import DirectRunner from apache_beam.runners.dask.dask_runner import DaskRunner from apache_beam.testing import test_pipeline +from apache_beam.testing.util import assert_that, equal_to class DaskRunnerRunPipelineTest(unittest.TestCase): @@ -28,8 +30,9 @@ def setUp(self) -> None: self.p = test_pipeline.TestPipeline(runner=DaskRunner()) def test_create(self): - _ = self.p | beam.Create([1]) - self.p.run() + with self.p as p: + pcoll = p | beam.Create([1]) + assert_that(pcoll, equal_to([1])) def test_create_and_map(self): def double(x): diff --git a/sdks/python/apache_beam/runners/dask/overrides.py b/sdks/python/apache_beam/runners/dask/overrides.py index 30df036eb7d9..ed88184a98f7 100644 --- a/sdks/python/apache_beam/runners/dask/overrides.py +++ b/sdks/python/apache_beam/runners/dask/overrides.py @@ -15,6 +15,7 @@ # limitations under the License. # import dataclasses +import time import typing as t import apache_beam as beam @@ -32,6 +33,8 @@ from apache_beam.runners.direct.direct_runner import _GroupAlsoByWindowDoFn from apache_beam.transforms import ptransform from apache_beam.transforms.window import GlobalWindows +from apache_beam.typehints import TypeCheckError +from apache_beam.utils.windowed_value import WindowedValue K = t.TypeVar("K") V = t.TypeVar("V") @@ -78,7 +81,7 @@ def infer_output_type(self, input_type): @typehints.with_input_types(t.Tuple[K, t.Iterable[V]]) @typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) class _GroupAlsoByWindow(ParDo): - """Not used yet...""" + def __init__(self, windowing): super(_GroupAlsoByWindow, self).__init__( _GroupAlsoByWindowDoFn(windowing)) @@ -91,13 +94,21 @@ def expand(self, input_or_inputs): @typehints.with_input_types(t.Tuple[K, V]) @typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) class _GroupByKey(PTransform): - """Not used yet...""" + def expand(self, input_or_inputs): return ( input_or_inputs - | "ReifyWindows" >> ParDo(GroupByKey.ReifyWindows()) + # | "ReifyWindows" >> ParDo(GroupByKey.ReifyWindows()) | "GroupByKey" >> _GroupByKeyOnly() - | "GroupByWindow" >> _GroupAlsoByWindow(input_or_inputs.windowing)) + # | "GetValue" >> beam.Map(lambda p: p[1]) + # | "GroupByWindow" >> _GroupAlsoByWindow(input_or_inputs.windowing) + ) + + +class _Flatten(PTransform): + def expand(self, input_or_inputs): + is_bounded = all(pcoll.is_bounded for pcoll in input_or_inputs) + return pvalue.PCollection(self.pipeline, is_bounded=is_bounded) def dask_overrides() -> t.List[PTransformOverride]: @@ -131,11 +142,20 @@ def matches(self, applied_ptransform: AppliedPTransform) -> bool: def get_replacement_transform_for_applied_ptransform( self, applied_ptransform: AppliedPTransform) -> ptransform.PTransform: - return _GroupByKeyOnly() + return _GroupByKey() + + class FlattenOverride(PTransformOverride): + def matches(self, applied_ptransform: AppliedPTransform) -> bool: + return applied_ptransform.transform.__class__ == beam.Flatten + + def get_replacement_transform_for_applied_ptransform( + self, applied_ptransform: AppliedPTransform) -> ptransform.PTransform: + return _Flatten() return [ CreateOverride(), ReshuffleOverride(), ReadOverride(), GroupByKeyOverride(), + FlattenOverride(), ] diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index e5b6ae00aba2..762db2d81044 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -18,16 +18,19 @@ TODO(alxr): Translate ops from https://docs.dask.org/en/latest/bag-api.html. """ -import typing as t import abc import dataclasses - -import apache_beam -from apache_beam.pipeline import AppliedPTransform +import typing as t import dask.bag as db -from apache_beam.runners.dask.overrides import _Create, _GroupByKeyOnly +import apache_beam +from apache_beam.pipeline import AppliedPTransform +from apache_beam.runners.dask.overrides import ( + _Create, + _GroupByKeyOnly, + _Flatten +) @dataclasses.dataclass @@ -76,9 +79,16 @@ def key(item): return input_bag.groupby(key) +class Flatten(DaskBagOp): + def apply(self, input_bag: t.Optional[t.Sequence[db.Bag]]) -> db.Bag: + assert type(input_bag) is list, 'Must take a sequence of bags!' + return db.concat(input_bag) + + TRANSLATIONS = { _Create: Create, apache_beam.ParDo: ParDo, apache_beam.Map: Map, _GroupByKeyOnly: GroupByKey, + _Flatten: Flatten, } From 2e3a126045b47c22433e4252993aca92ff8b7937 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Fri, 8 Jul 2022 15:34:03 -0700 Subject: [PATCH 047/173] With a great 1-liner from @pabloem, groupby is fixed! Now, all three initial tests pass. --- .../apache_beam/runners/dask/dask_runner_test.py | 16 ++++++++++------ .../runners/dask/transform_evaluator.py | 6 +++++- 2 files changed, 15 insertions(+), 7 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index a3788348336d..fcfc59f59bfa 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -27,10 +27,10 @@ class DaskRunnerRunPipelineTest(unittest.TestCase): """Test class used to introspect the dask runner via a debugger.""" def setUp(self) -> None: - self.p = test_pipeline.TestPipeline(runner=DaskRunner()) + self.pipeline = test_pipeline.TestPipeline(runner=DaskRunner()) def test_create(self): - with self.p as p: + with self.pipeline as p: pcoll = p | beam.Create([1]) assert_that(pcoll, equal_to([1])) @@ -38,15 +38,19 @@ def test_create_and_map(self): def double(x): return x * 2 - _ = self.p | beam.Create([1]) | beam.Map(double) - self.p.run() + with self.pipeline as p: + pcoll = p | beam.Create([1]) | beam.Map(double) + assert_that(pcoll, equal_to([2])) def test_create_map_and_groupby(self): def double(x): return x * 2, x - _ = self.p | beam.Create([1]) | beam.Map(double) | beam.GroupByKey() - self.p.run() + with self.pipeline as p: + pcoll = p | beam.Create([1]) | beam.Map(double) | beam.GroupByKey() + assert_that(pcoll, equal_to([ + (2, [1]) + ])) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 762db2d81044..4f1f02d22e94 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -76,7 +76,11 @@ def apply(self, input_bag: t.Optional[db.Bag]) -> db.Bag: def key(item): return item[0] - return input_bag.groupby(key) + def value(item): + k, v = item + return k, [elm[1] for elm in v] + + return input_bag.groupby(key).map(value) class Flatten(DaskBagOp): From 6467b0e65f0ca774c3abeda7c6730cb029b94e03 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Fri, 8 Jul 2022 15:40:15 -0700 Subject: [PATCH 048/173] Self-review: Cleaned up dask runner impl. --- .../apache_beam/options/pipeline_options.py | 1 - sdks/python/apache_beam/runners/dask/__init__.py | 16 ++++++++++++++++ .../apache_beam/runners/dask/dask_runner.py | 10 ---------- .../apache_beam/runners/dask/dask_runner_test.py | 1 - .../python/apache_beam/runners/dask/overrides.py | 8 +------- .../runners/dask/transform_evaluator.py | 16 +++++++++------- 6 files changed, 26 insertions(+), 26 deletions(-) diff --git a/sdks/python/apache_beam/options/pipeline_options.py b/sdks/python/apache_beam/options/pipeline_options.py index 4d545372041c..7fa9396ed2e9 100644 --- a/sdks/python/apache_beam/options/pipeline_options.py +++ b/sdks/python/apache_beam/options/pipeline_options.py @@ -455,7 +455,6 @@ class StandardOptions(PipelineOptions): 'apache_beam.runners.portability.spark_runner.SparkRunner', 'apache_beam.runners.test.TestDirectRunner', 'apache_beam.runners.test.TestDataflowRunner', - 'apache_beam.runners.dask.dask_runner.DaskRunner', ) KNOWN_RUNNER_NAMES = [path.split('.')[-1] for path in ALL_KNOWN_RUNNERS] diff --git a/sdks/python/apache_beam/runners/dask/__init__.py b/sdks/python/apache_beam/runners/dask/__init__.py index e69de29bb2d1..cce3acad34a4 100644 --- a/sdks/python/apache_beam/runners/dask/__init__.py +++ b/sdks/python/apache_beam/runners/dask/__init__.py @@ -0,0 +1,16 @@ +# +# 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. +# diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index 4caab2cd475f..e77d223f178c 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -19,16 +19,6 @@ The DaskRunner is a runner implementation that executes a graph of transformations across processes and workers via Dask distributed's scheduler. - - -Ideas to explore / Notes: -- Write a PCollection subclass that wraps a Dask Bag. - - Would be the input + return of the translation operators. -- The Ray runner is more focused on Task scheduling; This should focus more on graph translation. - -- A bundle is a subset of elements in a PCollection. i.e. a small set of elements that are processed together. -- In Dask, it is probably the same as a partition. Thus, we probably don't need to worry about it; Dask should take - care of it. """ import argparse import dataclasses diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index fcfc59f59bfa..c731a8dae51d 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -17,7 +17,6 @@ import unittest import apache_beam as beam -from apache_beam.runners import DirectRunner from apache_beam.runners.dask.dask_runner import DaskRunner from apache_beam.testing import test_pipeline from apache_beam.testing.util import assert_that, equal_to diff --git a/sdks/python/apache_beam/runners/dask/overrides.py b/sdks/python/apache_beam/runners/dask/overrides.py index ed88184a98f7..3b172535bfe2 100644 --- a/sdks/python/apache_beam/runners/dask/overrides.py +++ b/sdks/python/apache_beam/runners/dask/overrides.py @@ -15,13 +15,11 @@ # limitations under the License. # import dataclasses -import time import typing as t import apache_beam as beam from apache_beam import ( Create, - GroupByKey, ParDo, PTransform, pvalue, @@ -33,8 +31,6 @@ from apache_beam.runners.direct.direct_runner import _GroupAlsoByWindowDoFn from apache_beam.transforms import ptransform from apache_beam.transforms.window import GlobalWindows -from apache_beam.typehints import TypeCheckError -from apache_beam.utils.windowed_value import WindowedValue K = t.TypeVar("K") V = t.TypeVar("V") @@ -81,6 +77,7 @@ def infer_output_type(self, input_type): @typehints.with_input_types(t.Tuple[K, t.Iterable[V]]) @typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) class _GroupAlsoByWindow(ParDo): + """Not used yet...""" def __init__(self, windowing): super(_GroupAlsoByWindow, self).__init__( @@ -98,10 +95,7 @@ class _GroupByKey(PTransform): def expand(self, input_or_inputs): return ( input_or_inputs - # | "ReifyWindows" >> ParDo(GroupByKey.ReifyWindows()) | "GroupByKey" >> _GroupByKeyOnly() - # | "GetValue" >> beam.Map(lambda p: p[1]) - # | "GroupByWindow" >> _GroupAlsoByWindow(input_or_inputs.windowing) ) diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 4f1f02d22e94..86817b0cbe95 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -32,6 +32,8 @@ _Flatten ) +OpInput = t.Union[db.Bag, t.Sequence[db.Bag], None] + @dataclasses.dataclass class DaskBagOp(abc.ABC): @@ -42,17 +44,17 @@ def side_inputs(self): return self.applied.side_inputs @abc.abstractmethod - def apply(self, input_bag: t.Optional[db.Bag]) -> db.Bag: + def apply(self, input_bag: OpInput) -> db.Bag: pass class NoOp(DaskBagOp): - def apply(self, input_bag: t.Optional[db.Bag]) -> db.Bag: + def apply(self, input_bag: OpInput) -> db.Bag: return input_bag class Create(DaskBagOp): - def apply(self, input_bag: t.Optional[db.Bag]) -> db.Bag: + def apply(self, input_bag: OpInput) -> db.Bag: assert input_bag is None, 'Create expects no input!' original_transform = t.cast(_Create, self.applied.transform) items = original_transform.values @@ -60,19 +62,19 @@ def apply(self, input_bag: t.Optional[db.Bag]) -> db.Bag: class ParDo(DaskBagOp): - def apply(self, input_bag: t.Optional[db.Bag]) -> db.Bag: + def apply(self, input_bag: OpInput) -> db.Bag: fn = t.cast(apache_beam.ParDo, self.applied.transform).fn return input_bag.map(fn.process).flatten() class Map(DaskBagOp): - def apply(self, input_bag: t.Optional[db.Bag]) -> db.Bag: + def apply(self, input_bag: OpInput) -> db.Bag: fn = t.cast(apache_beam.Map, self.applied.transform).fn return input_bag.map(fn.process) class GroupByKey(DaskBagOp): - def apply(self, input_bag: t.Optional[db.Bag]) -> db.Bag: + def apply(self, input_bag: OpInput) -> db.Bag: def key(item): return item[0] @@ -84,7 +86,7 @@ def value(item): class Flatten(DaskBagOp): - def apply(self, input_bag: t.Optional[t.Sequence[db.Bag]]) -> db.Bag: + def apply(self, input_bag: OpInput) -> db.Bag: assert type(input_bag) is list, 'Must take a sequence of bags!' return db.concat(input_bag) From 793ba86930e6e01daa79cdfb7234b69039cfc9b1 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Fri, 8 Jul 2022 15:48:00 -0700 Subject: [PATCH 049/173] Self-review: Remove TODOs, delete commented out code, other cleanup. --- .../apache_beam/runners/dask/dask_runner.py | 14 ------- .../apache_beam/runners/dask/overrides.py | 37 ++++++++----------- .../runners/dask/transform_evaluator.py | 6 ++- 3 files changed, 19 insertions(+), 38 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index e77d223f178c..088cd227c2e0 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -145,7 +145,6 @@ def run_pipeline(self, pipeline, options): except ImportError: raise ImportError('DaskRunner is not available. Please install apache_beam[dask].') - # TODO(alxr): Actually use this right. dask_options = options.view_as(DaskOptions).get_all_options(drop_default=True) client = ddist.Client(**dask_options) @@ -157,16 +156,3 @@ def run_pipeline(self, pipeline, options): futures = client.compute(list(dask_visitor.bags.values())) return DaskRunnerResult(client, futures) - - # if pipeline: - # pass - # else: - # raise ValueError('Proto or FunctionAPI environments are not supported.') - # if pipeline: - # - # # Flatten / Optimize graph? - # - # # Trigger a traversal of all reachable nodes. - # self.visit_transforms(pipeline, options) - # - # Get API Client? diff --git a/sdks/python/apache_beam/runners/dask/overrides.py b/sdks/python/apache_beam/runners/dask/overrides.py index 3b172535bfe2..008b7d5d92e8 100644 --- a/sdks/python/apache_beam/runners/dask/overrides.py +++ b/sdks/python/apache_beam/runners/dask/overrides.py @@ -18,13 +18,6 @@ import typing as t import apache_beam as beam -from apache_beam import ( - Create, - ParDo, - PTransform, - pvalue, - Windowing, -) from apache_beam import typehints from apache_beam.io.iobase import SourceBase from apache_beam.pipeline import PTransformOverride, AppliedPTransform @@ -37,37 +30,37 @@ @dataclasses.dataclass -class _Create(PTransform): +class _Create(beam.PTransform): values: t.Tuple[t.Any] def expand(self, input_or_inputs): - return pvalue.PCollection.from_(input_or_inputs) + return beam.pvalue.PCollection.from_(input_or_inputs) def get_windowing(self, inputs): # type: (typing.Any) -> Windowing - return Windowing(GlobalWindows()) + return beam.Windowing(GlobalWindows()) @typehints.with_input_types(K) @typehints.with_output_types(K) -class _Reshuffle(PTransform): +class _Reshuffle(beam.PTransform): def expand(self, input_or_inputs): - return pvalue.PCollection.from_(input_or_inputs) + return beam.pvalue.PCollection.from_(input_or_inputs) @dataclasses.dataclass -class _Read(PTransform): +class _Read(beam.PTransform): source: SourceBase def expand(self, input_or_inputs): - return pvalue.PCollection.from_(input_or_inputs) + return beam.pvalue.PCollection.from_(input_or_inputs) @typehints.with_input_types(t.Tuple[K, V]) @typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) -class _GroupByKeyOnly(PTransform): +class _GroupByKeyOnly(beam.PTransform): def expand(self, input_or_inputs): - return pvalue.PCollection.from_(input_or_inputs) + return beam.pvalue.PCollection.from_(input_or_inputs) def infer_output_type(self, input_type): key_type, value_type = typehints.trivial_inference.key_value_types(input_type) @@ -76,7 +69,7 @@ def infer_output_type(self, input_type): @typehints.with_input_types(t.Tuple[K, t.Iterable[V]]) @typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) -class _GroupAlsoByWindow(ParDo): +class _GroupAlsoByWindow(beam.ParDo): """Not used yet...""" def __init__(self, windowing): @@ -85,12 +78,12 @@ def __init__(self, windowing): self.windowing = windowing def expand(self, input_or_inputs): - return pvalue.PCollection.from_(input_or_inputs) + return beam.pvalue.PCollection.from_(input_or_inputs) @typehints.with_input_types(t.Tuple[K, V]) @typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) -class _GroupByKey(PTransform): +class _GroupByKey(beam.PTransform): def expand(self, input_or_inputs): return ( @@ -99,10 +92,10 @@ def expand(self, input_or_inputs): ) -class _Flatten(PTransform): +class _Flatten(beam.PTransform): def expand(self, input_or_inputs): is_bounded = all(pcoll.is_bounded for pcoll in input_or_inputs) - return pvalue.PCollection(self.pipeline, is_bounded=is_bounded) + return beam.pvalue.PCollection(self.pipeline, is_bounded=is_bounded) def dask_overrides() -> t.List[PTransformOverride]: @@ -112,7 +105,7 @@ def matches(self, applied_ptransform: AppliedPTransform) -> bool: def get_replacement_transform_for_applied_ptransform( self, applied_ptransform: AppliedPTransform) -> ptransform.PTransform: - return _Create(t.cast(Create, applied_ptransform.transform).values) + return _Create(t.cast(beam.Create, applied_ptransform.transform).values) class ReshuffleOverride(PTransformOverride): def matches(self, applied_ptransform: AppliedPTransform) -> bool: diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 86817b0cbe95..cada317aa0db 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -14,10 +14,12 @@ # See the License for the specific language governing permissions and # limitations under the License. # -""" +"""Transform Beam PTransforms into Dask Bag operations. -TODO(alxr): Translate ops from https://docs.dask.org/en/latest/bag-api.html. +A minimum set of operation substitutions, to adap Beam's PTransform model +to Dask Bag functions. """ +# TODO(alxr): Translate ops from https://docs.dask.org/en/latest/bag-api.html. import abc import dataclasses import typing as t From e5357921ad5b12820636958b39754451e8147933 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 9 Jul 2022 00:07:39 -0700 Subject: [PATCH 050/173] First pass at linting rules. --- .../apache_beam/runners/dask/dask_runner.py | 217 ++++++++++-------- .../runners/dask/dask_runner_test.py | 47 ++-- .../apache_beam/runners/dask/overrides.py | 161 ++++++------- .../runners/dask/transform_evaluator.py | 85 ++++--- 4 files changed, 263 insertions(+), 247 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index 088cd227c2e0..6fd1af8ceeb8 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -20,139 +20,152 @@ transformations across processes and workers via Dask distributed's scheduler. """ -import argparse import dataclasses -import typing as t -import dask.distributed +import argparse +import typing as t from apache_beam import pvalue from apache_beam.options.pipeline_options import PipelineOptions -from apache_beam.pipeline import PipelineVisitor, AppliedPTransform +from apache_beam.pipeline import AppliedPTransform +from apache_beam.pipeline import PipelineVisitor from apache_beam.runners.dask.overrides import dask_overrides -from apache_beam.runners.dask.transform_evaluator import TRANSLATIONS, NoOp +from apache_beam.runners.dask.transform_evaluator import TRANSLATIONS +from apache_beam.runners.dask.transform_evaluator import NoOp from apache_beam.runners.direct.direct_runner import BundleBasedDirectRunner -from apache_beam.runners.runner import PipelineResult, PipelineState +from apache_beam.runners.runner import PipelineResult +from apache_beam.runners.runner import PipelineState from apache_beam.utils.interactive_utils import is_in_notebook class DaskOptions(PipelineOptions): - @staticmethod - def _parse_timeout(candidate): - try: - return int(candidate) - except: # noqa - import dask - return dask.config.no_default - - @classmethod - def _add_argparse_args(cls, parser: argparse.ArgumentParser) -> None: - parser.add_argument('--dask_client_address', type=str, default=None, - help='Address of a dask Scheduler server. Will default to a `dask.LocalCluster()`.') - parser.add_argument('--dask_connection_timeout', dest='timeout', type=DaskOptions._parse_timeout, - help='Timeout duration for initial connection to the scheduler.') - parser.add_argument('--dask_scheduler_file', type=str, default=None, - help='Path to a file with scheduler information if available.') - # TODO(alxr): Add options for security. - parser.add_argument('--dask_client_name', dest='name', type=str, default=None, - help='Gives the client a name that will be included in logs generated on the scheduler ' - 'for matters relating to this client.') - parser.add_argument('--dask_connection_limit', dest='connection_limit', type=int, default=512, - help='The number of open comms to maintain at once in the connection pool.') + @staticmethod + def _parse_timeout(candidate): + try: + return int(candidate) + except (TypeError, ValueError): + import dask + return dask.config.no_default + + @classmethod + def _add_argparse_args(cls, parser: argparse.ArgumentParser) -> None: + parser.add_argument('--dask_client_address', type=str, default=None, + help='Address of a dask Scheduler server. Will ' + 'default to a `dask.LocalCluster()`.') + parser.add_argument('--dask_connection_timeout', dest='timeout', + type=DaskOptions._parse_timeout, + help='Timeout duration for initial connection to the ' + 'scheduler.') + parser.add_argument('--dask_scheduler_file', type=str, default=None, + help='Path to a file with scheduler information if ' + 'available.') + # TODO(alxr): Add options for security. + parser.add_argument('--dask_client_name', dest='name', type=str, + default=None, + help='Gives the client a name that will be included ' + 'in logs generated on the scheduler for matters ' + 'relating to this client.') + parser.add_argument('--dask_connection_limit', dest='connection_limit', + type=int, default=512, + help='The number of open comms to maintain at once in ' + 'the connection pool.') @dataclasses.dataclass class DaskRunnerResult(PipelineResult): - client: 'dask.distributed.Client' - futures: t.Sequence['dask.distributed.Future'] - - def __post_init__(self): - super().__init__(PipelineState.RUNNING) - - def wait_until_finish(self, duration=None) -> PipelineState: - try: - if duration is not None: - # Convert milliseconds to seconds - duration /= 1000 - self.client.wait_for_workers(timeout=duration) - self.client.gather(self.futures, errors='raise') - self._state = PipelineState.DONE - except: # pylint: disable=broad-except - self._state = PipelineState.FAILED - raise - return self._state - - def cancel(self) -> PipelineState: - self._state = PipelineState.CANCELLING - self.client.cancel(self.futures) - self._state = PipelineState.CANCELLED - return self._state - - def metrics(self): - # TODO(alxr): Collect and return metrics... - raise NotImplementedError('collecting metrics will come later!') + from dask import distributed + + client: distributed.Client + futures: t.Sequence[distributed.Future] + + def __post_init__(self): + super().__init__(PipelineState.RUNNING) + + def wait_until_finish(self, duration=None) -> PipelineState: + try: + if duration is not None: + # Convert milliseconds to seconds + duration /= 1000 + self.client.wait_for_workers(timeout=duration) + self.client.gather(self.futures, errors='raise') + self._state = PipelineState.DONE + except: # pylint: disable=broad-except + self._state = PipelineState.FAILED + raise + return self._state + + def cancel(self) -> PipelineState: + self._state = PipelineState.CANCELLING + self.client.cancel(self.futures) + self._state = PipelineState.CANCELLED + return self._state + + def metrics(self): + # TODO(alxr): Collect and return metrics... + raise NotImplementedError('collecting metrics will come later!') class DaskRunner(BundleBasedDirectRunner): - """Executes a pipeline on a Dask distributed client.""" - - @staticmethod - def to_dask_bag_visitor() -> 'DaskBagVisitor': - from dask import bag as db + """Executes a pipeline on a Dask distributed client.""" - @dataclasses.dataclass - class DaskBagVisitor(PipelineVisitor): - bags: t.Dict[AppliedPTransform, db.Bag] = dataclasses.field(default_factory=dict) + @staticmethod + def to_dask_bag_visitor() -> PipelineVisitor: + from dask import bag as db - def visit_transform(self, transform_node: AppliedPTransform) -> None: - op_class = TRANSLATIONS.get(transform_node.transform.__class__, NoOp) - op = op_class(transform_node) + @dataclasses.dataclass + class DaskBagVisitor(PipelineVisitor): + bags: t.Dict[AppliedPTransform, db.Bag] = dataclasses.field( + default_factory=dict) - inputs = list(transform_node.inputs) - if inputs: - bag_inputs = [] - for input_value in inputs: - if isinstance(input_value, pvalue.PBegin): - bag_inputs.append(None) + def visit_transform(self, transform_node: AppliedPTransform) -> None: + op_class = TRANSLATIONS.get(transform_node.transform.__class__, NoOp) + op = op_class(transform_node) - prev_op = input_value.producer - if prev_op in self.bags: - bag_inputs.append(self.bags[prev_op]) + inputs = list(transform_node.inputs) + if inputs: + bag_inputs = [] + for input_value in inputs: + if isinstance(input_value, pvalue.PBegin): + bag_inputs.append(None) - if len(bag_inputs) == 1: - self.bags[transform_node] = op.apply(bag_inputs[0]) - else: - self.bags[transform_node] = op.apply(bag_inputs) + prev_op = input_value.producer + if prev_op in self.bags: + bag_inputs.append(self.bags[prev_op]) - else: - self.bags[transform_node] = op.apply(None) + if len(bag_inputs) == 1: + self.bags[transform_node] = op.apply(bag_inputs[0]) + else: + self.bags[transform_node] = op.apply(bag_inputs) - return DaskBagVisitor() + else: + self.bags[transform_node] = op.apply(None) - @staticmethod - def is_fnapi_compatible(): - return False + return DaskBagVisitor() - def run_pipeline(self, pipeline, options): - # TODO(alxr): Create interactive notebook support. - if is_in_notebook(): - raise NotImplementedError('interactive support will come later!') + @staticmethod + def is_fnapi_compatible(): + return False - try: - import dask.bag as db - import dask.distributed as ddist - except ImportError: - raise ImportError('DaskRunner is not available. Please install apache_beam[dask].') + def run_pipeline(self, pipeline, options): + # TODO(alxr): Create interactive notebook support. + if is_in_notebook(): + raise NotImplementedError('interactive support will come later!') - dask_options = options.view_as(DaskOptions).get_all_options(drop_default=True) - client = ddist.Client(**dask_options) + try: + import dask.distributed as ddist + except ImportError: + raise ImportError( + 'DaskRunner is not available. Please install apache_beam[dask].') - pipeline.replace_all(dask_overrides()) + dask_options = options.view_as(DaskOptions).get_all_options( + drop_default=True) + client = ddist.Client(**dask_options) - dask_visitor = self.to_dask_bag_visitor() - pipeline.visit(dask_visitor) + pipeline.replace_all(dask_overrides()) - futures = client.compute(list(dask_visitor.bags.values())) + dask_visitor = self.to_dask_bag_visitor() + pipeline.visit(dask_visitor) - return DaskRunnerResult(client, futures) + futures = client.compute(list(dask_visitor.bags.values())) + return DaskRunnerResult(client, futures) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index c731a8dae51d..e5a38dbbd83e 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -19,38 +19,39 @@ import apache_beam as beam from apache_beam.runners.dask.dask_runner import DaskRunner from apache_beam.testing import test_pipeline -from apache_beam.testing.util import assert_that, equal_to +from apache_beam.testing.util import assert_that +from apache_beam.testing.util import equal_to class DaskRunnerRunPipelineTest(unittest.TestCase): - """Test class used to introspect the dask runner via a debugger.""" + """Test class used to introspect the dask runner via a debugger.""" - def setUp(self) -> None: - self.pipeline = test_pipeline.TestPipeline(runner=DaskRunner()) + def setUp(self) -> None: + self.pipeline = test_pipeline.TestPipeline(runner=DaskRunner()) - def test_create(self): - with self.pipeline as p: - pcoll = p | beam.Create([1]) - assert_that(pcoll, equal_to([1])) + def test_create(self): + with self.pipeline as p: + pcoll = p | beam.Create([1]) + assert_that(pcoll, equal_to([1])) - def test_create_and_map(self): - def double(x): - return x * 2 + def test_create_and_map(self): + def double(x): + return x * 2 - with self.pipeline as p: - pcoll = p | beam.Create([1]) | beam.Map(double) - assert_that(pcoll, equal_to([2])) + with self.pipeline as p: + pcoll = p | beam.Create([1]) | beam.Map(double) + assert_that(pcoll, equal_to([2])) - def test_create_map_and_groupby(self): - def double(x): - return x * 2, x + def test_create_map_and_groupby(self): + def double(x): + return x * 2, x - with self.pipeline as p: - pcoll = p | beam.Create([1]) | beam.Map(double) | beam.GroupByKey() - assert_that(pcoll, equal_to([ - (2, [1]) - ])) + with self.pipeline as p: + pcoll = p | beam.Create([1]) | beam.Map(double) | beam.GroupByKey() + assert_that(pcoll, equal_to([ + (2, [1]) + ])) if __name__ == '__main__': - unittest.main() + unittest.main() diff --git a/sdks/python/apache_beam/runners/dask/overrides.py b/sdks/python/apache_beam/runners/dask/overrides.py index 008b7d5d92e8..f06b6de6a10e 100644 --- a/sdks/python/apache_beam/runners/dask/overrides.py +++ b/sdks/python/apache_beam/runners/dask/overrides.py @@ -15,12 +15,14 @@ # limitations under the License. # import dataclasses + import typing as t import apache_beam as beam from apache_beam import typehints from apache_beam.io.iobase import SourceBase -from apache_beam.pipeline import PTransformOverride, AppliedPTransform +from apache_beam.pipeline import AppliedPTransform +from apache_beam.pipeline import PTransformOverride from apache_beam.runners.direct.direct_runner import _GroupAlsoByWindowDoFn from apache_beam.transforms import ptransform from apache_beam.transforms.window import GlobalWindows @@ -31,118 +33,119 @@ @dataclasses.dataclass class _Create(beam.PTransform): - values: t.Tuple[t.Any] + values: t.Tuple[t.Any] - def expand(self, input_or_inputs): - return beam.pvalue.PCollection.from_(input_or_inputs) + def expand(self, input_or_inputs): + return beam.pvalue.PCollection.from_(input_or_inputs) - def get_windowing(self, inputs): - # type: (typing.Any) -> Windowing - return beam.Windowing(GlobalWindows()) + def get_windowing(self, inputs: t.Any) -> beam.Windowing: + return beam.Windowing(GlobalWindows()) @typehints.with_input_types(K) @typehints.with_output_types(K) class _Reshuffle(beam.PTransform): - def expand(self, input_or_inputs): - return beam.pvalue.PCollection.from_(input_or_inputs) + def expand(self, input_or_inputs): + return beam.pvalue.PCollection.from_(input_or_inputs) @dataclasses.dataclass class _Read(beam.PTransform): - source: SourceBase + source: SourceBase - def expand(self, input_or_inputs): - return beam.pvalue.PCollection.from_(input_or_inputs) + def expand(self, input_or_inputs): + return beam.pvalue.PCollection.from_(input_or_inputs) @typehints.with_input_types(t.Tuple[K, V]) @typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) class _GroupByKeyOnly(beam.PTransform): - def expand(self, input_or_inputs): - return beam.pvalue.PCollection.from_(input_or_inputs) + def expand(self, input_or_inputs): + return beam.pvalue.PCollection.from_(input_or_inputs) + + def infer_output_type(self, input_type): - def infer_output_type(self, input_type): - key_type, value_type = typehints.trivial_inference.key_value_types(input_type) - return typehints.KV[key_type, typehints.Iterable[value_type]] + key_type, value_type = typehints.trivial_inference.key_value_types( + input_type + ) + return typehints.KV[key_type, typehints.Iterable[value_type]] @typehints.with_input_types(t.Tuple[K, t.Iterable[V]]) @typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) class _GroupAlsoByWindow(beam.ParDo): - """Not used yet...""" + """Not used yet...""" - def __init__(self, windowing): - super(_GroupAlsoByWindow, self).__init__( - _GroupAlsoByWindowDoFn(windowing)) - self.windowing = windowing + def __init__(self, windowing): + super().__init__(_GroupAlsoByWindowDoFn(windowing)) + self.windowing = windowing - def expand(self, input_or_inputs): - return beam.pvalue.PCollection.from_(input_or_inputs) + def expand(self, input_or_inputs): + return beam.pvalue.PCollection.from_(input_or_inputs) @typehints.with_input_types(t.Tuple[K, V]) @typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) class _GroupByKey(beam.PTransform): - def expand(self, input_or_inputs): - return ( - input_or_inputs - | "GroupByKey" >> _GroupByKeyOnly() - ) + def expand(self, input_or_inputs): + return ( + input_or_inputs + | "GroupByKey" >> _GroupByKeyOnly() + ) class _Flatten(beam.PTransform): - def expand(self, input_or_inputs): - is_bounded = all(pcoll.is_bounded for pcoll in input_or_inputs) - return beam.pvalue.PCollection(self.pipeline, is_bounded=is_bounded) + def expand(self, input_or_inputs): + is_bounded = all(pcoll.is_bounded for pcoll in input_or_inputs) + return beam.pvalue.PCollection(self.pipeline, is_bounded=is_bounded) def dask_overrides() -> t.List[PTransformOverride]: - class CreateOverride(PTransformOverride): - def matches(self, applied_ptransform: AppliedPTransform) -> bool: - return applied_ptransform.transform.__class__ == beam.Create - - def get_replacement_transform_for_applied_ptransform( - self, applied_ptransform: AppliedPTransform) -> ptransform.PTransform: - return _Create(t.cast(beam.Create, applied_ptransform.transform).values) - - class ReshuffleOverride(PTransformOverride): - def matches(self, applied_ptransform: AppliedPTransform) -> bool: - return applied_ptransform.transform.__class__ == beam.Reshuffle - - def get_replacement_transform_for_applied_ptransform( - self, applied_ptransform: AppliedPTransform) -> ptransform.PTransform: - return _Reshuffle() - - class ReadOverride(PTransformOverride): - def matches(self, applied_ptransform: AppliedPTransform) -> bool: - return applied_ptransform.transform.__class__ == beam.io.Read - - def get_replacement_transform_for_applied_ptransform( - self, applied_ptransform: AppliedPTransform) -> ptransform.PTransform: - return _Read(t.cast(beam.io.Read, applied_ptransform.transform).source) - - class GroupByKeyOverride(PTransformOverride): - def matches(self, applied_ptransform: AppliedPTransform) -> bool: - return applied_ptransform.transform.__class__ == beam.GroupByKey - - def get_replacement_transform_for_applied_ptransform( - self, applied_ptransform: AppliedPTransform) -> ptransform.PTransform: - return _GroupByKey() - - class FlattenOverride(PTransformOverride): - def matches(self, applied_ptransform: AppliedPTransform) -> bool: - return applied_ptransform.transform.__class__ == beam.Flatten - - def get_replacement_transform_for_applied_ptransform( - self, applied_ptransform: AppliedPTransform) -> ptransform.PTransform: - return _Flatten() - - return [ - CreateOverride(), - ReshuffleOverride(), - ReadOverride(), - GroupByKeyOverride(), - FlattenOverride(), - ] + class CreateOverride(PTransformOverride): + def matches(self, applied_ptransform: AppliedPTransform) -> bool: + return applied_ptransform.transform.__class__ == beam.Create + + def get_replacement_transform_for_applied_ptransform( + self, applied_ptransform: AppliedPTransform) -> ptransform.PTransform: + return _Create(t.cast(beam.Create, applied_ptransform.transform).values) + + class ReshuffleOverride(PTransformOverride): + def matches(self, applied_ptransform: AppliedPTransform) -> bool: + return applied_ptransform.transform.__class__ == beam.Reshuffle + + def get_replacement_transform_for_applied_ptransform( + self, applied_ptransform: AppliedPTransform) -> ptransform.PTransform: + return _Reshuffle() + + class ReadOverride(PTransformOverride): + def matches(self, applied_ptransform: AppliedPTransform) -> bool: + return applied_ptransform.transform.__class__ == beam.io.Read + + def get_replacement_transform_for_applied_ptransform( + self, applied_ptransform: AppliedPTransform) -> ptransform.PTransform: + return _Read(t.cast(beam.io.Read, applied_ptransform.transform).source) + + class GroupByKeyOverride(PTransformOverride): + def matches(self, applied_ptransform: AppliedPTransform) -> bool: + return applied_ptransform.transform.__class__ == beam.GroupByKey + + def get_replacement_transform_for_applied_ptransform( + self, applied_ptransform: AppliedPTransform) -> ptransform.PTransform: + return _GroupByKey() + + class FlattenOverride(PTransformOverride): + def matches(self, applied_ptransform: AppliedPTransform) -> bool: + return applied_ptransform.transform.__class__ == beam.Flatten + + def get_replacement_transform_for_applied_ptransform( + self, applied_ptransform: AppliedPTransform) -> ptransform.PTransform: + return _Flatten() + + return [ + CreateOverride(), + ReshuffleOverride(), + ReadOverride(), + GroupByKeyOverride(), + FlattenOverride(), + ] diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index cada317aa0db..ae58a8bae414 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -18,85 +18,84 @@ A minimum set of operation substitutions, to adap Beam's PTransform model to Dask Bag functions. + +TODO(alxr): Translate ops from https://docs.dask.org/en/latest/bag-api.html. """ -# TODO(alxr): Translate ops from https://docs.dask.org/en/latest/bag-api.html. -import abc import dataclasses -import typing as t +import abc import dask.bag as db +import typing as t import apache_beam from apache_beam.pipeline import AppliedPTransform -from apache_beam.runners.dask.overrides import ( - _Create, - _GroupByKeyOnly, - _Flatten -) +from apache_beam.runners.dask.overrides import _Create +from apache_beam.runners.dask.overrides import _Flatten +from apache_beam.runners.dask.overrides import _GroupByKeyOnly OpInput = t.Union[db.Bag, t.Sequence[db.Bag], None] @dataclasses.dataclass class DaskBagOp(abc.ABC): - applied: AppliedPTransform + applied: AppliedPTransform - @property - def side_inputs(self): - return self.applied.side_inputs + @property + def side_inputs(self): + return self.applied.side_inputs - @abc.abstractmethod - def apply(self, input_bag: OpInput) -> db.Bag: - pass + @abc.abstractmethod + def apply(self, input_bag: OpInput) -> db.Bag: + pass class NoOp(DaskBagOp): - def apply(self, input_bag: OpInput) -> db.Bag: - return input_bag + def apply(self, input_bag: OpInput) -> db.Bag: + return input_bag class Create(DaskBagOp): - def apply(self, input_bag: OpInput) -> db.Bag: - assert input_bag is None, 'Create expects no input!' - original_transform = t.cast(_Create, self.applied.transform) - items = original_transform.values - return db.from_sequence(items) + def apply(self, input_bag: OpInput) -> db.Bag: + assert input_bag is None, 'Create expects no input!' + original_transform = t.cast(_Create, self.applied.transform) + items = original_transform.values + return db.from_sequence(items) class ParDo(DaskBagOp): - def apply(self, input_bag: OpInput) -> db.Bag: - fn = t.cast(apache_beam.ParDo, self.applied.transform).fn - return input_bag.map(fn.process).flatten() + def apply(self, input_bag: OpInput) -> db.Bag: + fn = t.cast(apache_beam.ParDo, self.applied.transform).fn + return input_bag.map(fn.process).flatten() class Map(DaskBagOp): - def apply(self, input_bag: OpInput) -> db.Bag: - fn = t.cast(apache_beam.Map, self.applied.transform).fn - return input_bag.map(fn.process) + def apply(self, input_bag: OpInput) -> db.Bag: + fn = t.cast(apache_beam.Map, self.applied.transform).fn + return input_bag.map(fn.process) class GroupByKey(DaskBagOp): - def apply(self, input_bag: OpInput) -> db.Bag: - def key(item): - return item[0] + def apply(self, input_bag: OpInput) -> db.Bag: + def key(item): + return item[0] - def value(item): - k, v = item - return k, [elm[1] for elm in v] + def value(item): + k, v = item + return k, [elm[1] for elm in v] - return input_bag.groupby(key).map(value) + return input_bag.groupby(key).map(value) class Flatten(DaskBagOp): - def apply(self, input_bag: OpInput) -> db.Bag: - assert type(input_bag) is list, 'Must take a sequence of bags!' - return db.concat(input_bag) + def apply(self, input_bag: OpInput) -> db.Bag: + assert type(input_bag) is list, 'Must take a sequence of bags!' + return db.concat(input_bag) TRANSLATIONS = { - _Create: Create, - apache_beam.ParDo: ParDo, - apache_beam.Map: Map, - _GroupByKeyOnly: GroupByKey, - _Flatten: Flatten, + _Create: Create, + apache_beam.ParDo: ParDo, + apache_beam.Map: Map, + _GroupByKeyOnly: GroupByKey, + _Flatten: Flatten, } From 8e326682463cdb3cf9c0d2fe466397e2ebd16df4 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 9 Jul 2022 00:18:58 -0700 Subject: [PATCH 051/173] WIP, include dask dependencies + test setup. --- sdks/python/setup.py | 3 ++- sdks/python/tox.ini | 6 +++++- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/sdks/python/setup.py b/sdks/python/setup.py index dc4dde575287..dcd382705a31 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -334,7 +334,8 @@ def get_portability_package_data(): 'azure-storage-blob >=12.3.2', 'azure-core >=1.7.0', ], - 'dataframe': ['pandas>=1.0,<1.5'] + 'dataframe': ['pandas>=1.0,<1.5'], + 'dask': ['dask[distributed] >= 2022.6'] }, zip_safe=False, # PyPI package information. diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini index ad9adc2149e6..95ca964ae00f 100644 --- a/sdks/python/tox.ini +++ b/sdks/python/tox.ini @@ -17,7 +17,7 @@ [tox] # new environments will be excluded by default unless explicitly added to envlist. -envlist = py37,py38,py39,py37-{cloud,cython,lint,mypy},py38-{cloud,cython,docs,cloudcoverage},py39-{cloud,cython},whitespacelint +envlist = py37,py38,py39,py37-{cloud,cython,lint,mypy,dask},py38-{cloud,cython,docs,cloudcoverage,dask},py39-{cloud,cython,dask},whitespacelint toxworkdir = {toxinidir}/target/{env:ENV_NAME:.tox} [pycodestyle] @@ -92,6 +92,10 @@ extras = test,gcp,interactive,dataframe,aws,azure commands = {toxinidir}/scripts/run_pytest.sh {envname} "{posargs}" +[testenv:py{37,38,39}-dask] +extras = test,dask +commands = + {toxinidir}/scripts/run_pytest.sh {envname} "{posargs}" [testenv:py38-cloudcoverage] deps = codecov From 318afc2e16ea06fa6d0fd9a96e5dec92648bc6a5 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 9 Jul 2022 00:34:11 -0700 Subject: [PATCH 052/173] WIP: maybe better dask deps? --- sdks/python/setup.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/sdks/python/setup.py b/sdks/python/setup.py index dcd382705a31..99b9825538a4 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -335,7 +335,10 @@ def get_portability_package_data(): 'azure-core >=1.7.0', ], 'dataframe': ['pandas>=1.0,<1.5'], - 'dask': ['dask[distributed] >= 2022.6'] + 'dask': [ + 'dask >= 2022.6' + 'distributed >= 2022.6' + ], }, zip_safe=False, # PyPI package information. From b01855fac5e2156d64fd7417668d81933754fb6f Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sun, 10 Jul 2022 16:41:59 -0700 Subject: [PATCH 053/173] Skip dask tests depending on successful import. --- sdks/python/apache_beam/runners/dask/dask_runner_test.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index e5a38dbbd83e..208ce77bace1 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -22,6 +22,11 @@ from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to +try: + import dask +except (ImportError, ModuleNotFoundError): + raise unittest.SkipTest('Dask must be installed to run tests.') + class DaskRunnerRunPipelineTest(unittest.TestCase): """Test class used to introspect the dask runner via a debugger.""" From 2c2eb8d06950b72e70af7225ad7ced8670f1f984 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sun, 10 Jul 2022 17:30:52 -0700 Subject: [PATCH 054/173] Fixed setup.py (missing `,`). --- sdks/python/setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/setup.py b/sdks/python/setup.py index 99b9825538a4..609817ffc02d 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -336,7 +336,7 @@ def get_portability_package_data(): ], 'dataframe': ['pandas>=1.0,<1.5'], 'dask': [ - 'dask >= 2022.6' + 'dask >= 2022.6', 'distributed >= 2022.6' ], }, From e64e9eb6e916f4a7a860ee0b5f9f09d245f05aa8 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sun, 10 Jul 2022 17:31:13 -0700 Subject: [PATCH 055/173] Added an additional comma. --- sdks/python/setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/setup.py b/sdks/python/setup.py index 609817ffc02d..70a377aabb7a 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -337,7 +337,7 @@ def get_portability_package_data(): 'dataframe': ['pandas>=1.0,<1.5'], 'dask': [ 'dask >= 2022.6', - 'distributed >= 2022.6' + 'distributed >= 2022.6', ], }, zip_safe=False, From 69b118b809be353ac6645d1498ba79115970a968 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sun, 10 Jul 2022 20:14:03 -0700 Subject: [PATCH 056/173] Moved skipping logic to be above dask import. --- .../apache_beam/runners/dask/dask_runner_test.py | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index 208ce77bace1..7e9041d7a67e 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -16,23 +16,24 @@ # import unittest +try: + import dask +except (ImportError, ModuleNotFoundError): + raise unittest.SkipTest('Dask must be installed to run tests.') + import apache_beam as beam from apache_beam.runners.dask.dask_runner import DaskRunner from apache_beam.testing import test_pipeline from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to -try: - import dask -except (ImportError, ModuleNotFoundError): - raise unittest.SkipTest('Dask must be installed to run tests.') class DaskRunnerRunPipelineTest(unittest.TestCase): """Test class used to introspect the dask runner via a debugger.""" def setUp(self) -> None: - self.pipeline = test_pipeline.TestPipeline(runner=DaskRunner()) + self.pipeline = test_pipeline.TestPipeline(runner=DaskRunner()) def test_create(self): with self.pipeline as p: From 9ffc8d8367e3b9780697f8799fe6ddce5556c16c Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Mon, 5 Sep 2022 15:03:34 -0700 Subject: [PATCH 057/173] Fix lint issues with dask runner tests. --- sdks/python/apache_beam/runners/dask/dask_runner_test.py | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index 7e9041d7a67e..53fa3fff4838 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -17,7 +17,7 @@ import unittest try: - import dask + import dask # pylint: disable=unused-import except (ImportError, ModuleNotFoundError): raise unittest.SkipTest('Dask must be installed to run tests.') @@ -28,12 +28,11 @@ from apache_beam.testing.util import equal_to - class DaskRunnerRunPipelineTest(unittest.TestCase): """Test class used to introspect the dask runner via a debugger.""" def setUp(self) -> None: - self.pipeline = test_pipeline.TestPipeline(runner=DaskRunner()) + self.pipeline = test_pipeline.TestPipeline(runner=DaskRunner()) def test_create(self): with self.pipeline as p: From 8a2afb71d145512627936fd8ebcf841f82a1fe92 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Mon, 19 Sep 2022 22:32:30 -0700 Subject: [PATCH 058/173] Adding destination for client address. --- sdks/python/apache_beam/runners/dask/dask_runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index 6fd1af8ceeb8..74915c4e254c 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -50,7 +50,7 @@ def _parse_timeout(candidate): @classmethod def _add_argparse_args(cls, parser: argparse.ArgumentParser) -> None: - parser.add_argument('--dask_client_address', type=str, default=None, + parser.add_argument('--dask_client_address', dest='address', type=str, default=None, help='Address of a dask Scheduler server. Will ' 'default to a `dask.LocalCluster()`.') parser.add_argument('--dask_connection_timeout', dest='timeout', From 93f02f1ec1511a4a6f6c65efc42ee1e21ccfce7b Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Wed, 21 Sep 2022 14:29:37 -0700 Subject: [PATCH 059/173] Changing to async produces a timeout error instead of stuck in infinite loop. --- sdks/python/apache_beam/runners/dask/dask_runner.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index 74915c4e254c..ed711ae00657 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -88,11 +88,13 @@ def wait_until_finish(self, duration=None) -> PipelineState: # Convert milliseconds to seconds duration /= 1000 self.client.wait_for_workers(timeout=duration) - self.client.gather(self.futures, errors='raise') + self.client.gather(self.futures, errors='raise', asynchronous=True) self._state = PipelineState.DONE except: # pylint: disable=broad-except self._state = PipelineState.FAILED raise + # finally: + # self.client.close(timeout=duration) return self._state def cancel(self) -> PipelineState: From afdcf1b184aeb1a5448e562fa5d0432f9341c779 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Thu, 22 Sep 2022 08:59:42 -0700 Subject: [PATCH 060/173] Close client during `wait_until_finish`; rm async. --- sdks/python/apache_beam/runners/dask/dask_runner.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index ed711ae00657..5b4c297052c0 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -88,13 +88,14 @@ def wait_until_finish(self, duration=None) -> PipelineState: # Convert milliseconds to seconds duration /= 1000 self.client.wait_for_workers(timeout=duration) - self.client.gather(self.futures, errors='raise', asynchronous=True) + self.client.gather(self.futures, errors='raise') self._state = PipelineState.DONE except: # pylint: disable=broad-except self._state = PipelineState.FAILED raise - # finally: - # self.client.close(timeout=duration) + finally: + self.client.close() + self.client.cluster.close() return self._state def cancel(self) -> PipelineState: From 41b52676f8bdd41be34e6ed1e0f166d1c87bc4a3 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 1 Oct 2022 21:24:14 -0400 Subject: [PATCH 061/173] Supporting side-inputs for ParDo. --- sdks/python/apache_beam/runners/dask/dask_runner_test.py | 9 +++++++++ .../apache_beam/runners/dask/transform_evaluator.py | 5 +++-- 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index 53fa3fff4838..bd9253e6bc2b 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -57,6 +57,15 @@ def double(x): (2, [1]) ])) + def test_map_with_side_inputs(self): + def mult_by(x, y=2): + return x * y + + with self.pipeline as p: + pcoll = p | beam.Create([1]) | beam.Map(mult_by, y=3) + assert_that(pcoll, equal_to([3])) + + if __name__ == '__main__': unittest.main() diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index ae58a8bae414..92c5922c344a 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -26,6 +26,7 @@ import abc import dask.bag as db import typing as t +import functools import apache_beam from apache_beam.pipeline import AppliedPTransform @@ -65,13 +66,13 @@ def apply(self, input_bag: OpInput) -> db.Bag: class ParDo(DaskBagOp): def apply(self, input_bag: OpInput) -> db.Bag: fn = t.cast(apache_beam.ParDo, self.applied.transform).fn - return input_bag.map(fn.process).flatten() + return input_bag.map(fn.process, *self.side_inputs).flatten() class Map(DaskBagOp): def apply(self, input_bag: OpInput) -> db.Bag: fn = t.cast(apache_beam.Map, self.applied.transform).fn - return input_bag.map(fn.process) + return input_bag.map(fn.process, *self.side_inputs) class GroupByKey(DaskBagOp): From e3ac3f8e8585f6ca95116a58e7d3a08ab3c91127 Mon Sep 17 00:00:00 2001 From: Pablo E Date: Wed, 28 Sep 2022 12:03:02 -0700 Subject: [PATCH 062/173] Revert "Close client during `wait_until_finish`; rm async." This reverts commit 09365f628dc86832190efd03a308ba177755bf13. --- sdks/python/apache_beam/runners/dask/dask_runner.py | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index 5b4c297052c0..ed711ae00657 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -88,14 +88,13 @@ def wait_until_finish(self, duration=None) -> PipelineState: # Convert milliseconds to seconds duration /= 1000 self.client.wait_for_workers(timeout=duration) - self.client.gather(self.futures, errors='raise') + self.client.gather(self.futures, errors='raise', asynchronous=True) self._state = PipelineState.DONE except: # pylint: disable=broad-except self._state = PipelineState.FAILED raise - finally: - self.client.close() - self.client.cluster.close() + # finally: + # self.client.close(timeout=duration) return self._state def cancel(self) -> PipelineState: From 3fddc814c46c885aebd1d202c563beac0991efc4 Mon Sep 17 00:00:00 2001 From: Pablo E Date: Wed, 28 Sep 2022 12:03:07 -0700 Subject: [PATCH 063/173] Revert "Changing to async produces a timeout error instead of stuck in infinite loop." This reverts commit 676d75226c3f517083baef5dab4a52e5cde0f1a0. --- sdks/python/apache_beam/runners/dask/dask_runner.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index ed711ae00657..74915c4e254c 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -88,13 +88,11 @@ def wait_until_finish(self, duration=None) -> PipelineState: # Convert milliseconds to seconds duration /= 1000 self.client.wait_for_workers(timeout=duration) - self.client.gather(self.futures, errors='raise', asynchronous=True) + self.client.gather(self.futures, errors='raise') self._state = PipelineState.DONE except: # pylint: disable=broad-except self._state = PipelineState.FAILED raise - # finally: - # self.client.close(timeout=duration) return self._state def cancel(self) -> PipelineState: From 9eeb9eaae3b4f86cbb66fd6870c753fbbb16dd55 Mon Sep 17 00:00:00 2001 From: Pablo E Date: Wed, 28 Sep 2022 12:04:21 -0700 Subject: [PATCH 064/173] Adding -dask tox targets onto the gradle build --- sdks/python/test-suites/tox/common.gradle | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sdks/python/test-suites/tox/common.gradle b/sdks/python/test-suites/tox/common.gradle index 99afc1d72557..61802ac9c45e 100644 --- a/sdks/python/test-suites/tox/common.gradle +++ b/sdks/python/test-suites/tox/common.gradle @@ -24,6 +24,9 @@ test.dependsOn "testPython${pythonVersionSuffix}" toxTask "testPy${pythonVersionSuffix}Cloud", "py${pythonVersionSuffix}-cloud" test.dependsOn "testPy${pythonVersionSuffix}Cloud" +toxTask "testPy${pythonVersionSuffix}Dask", "py${pythonVersionSuffix}-dask" +test.dependsOn "testPy${pythonVersionSuffix}Dask" + toxTask "testPy${pythonVersionSuffix}Cython", "py${pythonVersionSuffix}-cython" test.dependsOn "testPy${pythonVersionSuffix}Cython" From b4d09990615d4eaf3e87ef74d2f0015a89cd89aa Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 1 Oct 2022 21:27:03 -0400 Subject: [PATCH 065/173] wip - added print stmt. --- sdks/python/apache_beam/runners/dask/transform_evaluator.py | 1 + 1 file changed, 1 insertion(+) diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 92c5922c344a..2ae7caa130f8 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -72,6 +72,7 @@ def apply(self, input_bag: OpInput) -> db.Bag: class Map(DaskBagOp): def apply(self, input_bag: OpInput) -> db.Bag: fn = t.cast(apache_beam.Map, self.applied.transform).fn + print(self.side_inputs) return input_bag.map(fn.process, *self.side_inputs) From 0319ffd2428b7bb9a53f937bebd072bb9db85615 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 1 Oct 2022 21:29:57 -0400 Subject: [PATCH 066/173] wip - prove side inputs is set. --- sdks/python/apache_beam/runners/dask/transform_evaluator.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 2ae7caa130f8..5d70a871fc40 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -72,7 +72,7 @@ def apply(self, input_bag: OpInput) -> db.Bag: class Map(DaskBagOp): def apply(self, input_bag: OpInput) -> db.Bag: fn = t.cast(apache_beam.Map, self.applied.transform).fn - print(self.side_inputs) + assert self.side_inputs is (3,) return input_bag.map(fn.process, *self.side_inputs) From 0b13bb07d86aaa890db92493f7c3a96fc70803df Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 1 Oct 2022 21:31:13 -0400 Subject: [PATCH 067/173] wip - prove side inputs is set in Pardo. --- sdks/python/apache_beam/runners/dask/transform_evaluator.py | 1 + 1 file changed, 1 insertion(+) diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 5d70a871fc40..7c24f180aef0 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -66,6 +66,7 @@ def apply(self, input_bag: OpInput) -> db.Bag: class ParDo(DaskBagOp): def apply(self, input_bag: OpInput) -> db.Bag: fn = t.cast(apache_beam.ParDo, self.applied.transform).fn + assert self.side_inputs is (3,) return input_bag.map(fn.process, *self.side_inputs).flatten() From 1e7052b06caa2e352bde14576ffd5bac18c4d00b Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 1 Oct 2022 21:32:27 -0400 Subject: [PATCH 068/173] wip - rm asserts, add print --- sdks/python/apache_beam/runners/dask/transform_evaluator.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 7c24f180aef0..408bc38cce52 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -66,14 +66,13 @@ def apply(self, input_bag: OpInput) -> db.Bag: class ParDo(DaskBagOp): def apply(self, input_bag: OpInput) -> db.Bag: fn = t.cast(apache_beam.ParDo, self.applied.transform).fn - assert self.side_inputs is (3,) + print(self.side_inputs) return input_bag.map(fn.process, *self.side_inputs).flatten() class Map(DaskBagOp): def apply(self, input_bag: OpInput) -> db.Bag: fn = t.cast(apache_beam.Map, self.applied.transform).fn - assert self.side_inputs is (3,) return input_bag.map(fn.process, *self.side_inputs) From 292e023a0e86eba21878a477a3a228fafe4e8881 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 1 Oct 2022 21:41:05 -0400 Subject: [PATCH 069/173] wip - adding named inputs... --- .../apache_beam/runners/dask/transform_evaluator.py | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 408bc38cce52..50f4ba51279f 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -45,6 +45,10 @@ class DaskBagOp(abc.ABC): def side_inputs(self): return self.applied.side_inputs + @property + def named_inputs(self): + return self.applied.named_inputs() + @abc.abstractmethod def apply(self, input_bag: OpInput) -> db.Bag: pass @@ -67,13 +71,14 @@ class ParDo(DaskBagOp): def apply(self, input_bag: OpInput) -> db.Bag: fn = t.cast(apache_beam.ParDo, self.applied.transform).fn print(self.side_inputs) - return input_bag.map(fn.process, *self.side_inputs).flatten() + print(self.named_inputs) + return input_bag.map(fn.process, *self.side_inputs, **self.named_inputs).flatten() class Map(DaskBagOp): def apply(self, input_bag: OpInput) -> db.Bag: fn = t.cast(apache_beam.Map, self.applied.transform).fn - return input_bag.map(fn.process, *self.side_inputs) + return input_bag.map(fn.process, *self.side_inputs, **self.named_inputs) class GroupByKey(DaskBagOp): From 31c1e2bed604ee0bde739dd130aaac513f2d0c49 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 1 Oct 2022 21:55:06 -0400 Subject: [PATCH 070/173] Experiments: non-named side inputs + del `None` in named inputs. --- sdks/python/apache_beam/runners/dask/dask_runner_test.py | 2 +- sdks/python/apache_beam/runners/dask/transform_evaluator.py | 6 ++++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index bd9253e6bc2b..6e69e90938a9 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -62,7 +62,7 @@ def mult_by(x, y=2): return x * y with self.pipeline as p: - pcoll = p | beam.Create([1]) | beam.Map(mult_by, y=3) + pcoll = p | beam.Create([1]) | beam.Map(mult_by, 3) assert_that(pcoll, equal_to([3])) diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 50f4ba51279f..b77e50e57c1e 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -45,9 +45,11 @@ class DaskBagOp(abc.ABC): def side_inputs(self): return self.applied.side_inputs - @property + @functools.cached_property def named_inputs(self): - return self.applied.named_inputs() + named_inputs = self.applied.named_inputs() + del named_inputs[None] + return named_inputs @abc.abstractmethod def apply(self, input_bag: OpInput) -> db.Bag: From f4ecf2f16d776671c892c2d8dc5f6354d3e6f2e5 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 1 Oct 2022 21:56:35 -0400 Subject: [PATCH 071/173] None --> 'None' --- sdks/python/apache_beam/runners/dask/transform_evaluator.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index b77e50e57c1e..3f4676c8f3e2 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -48,7 +48,7 @@ def side_inputs(self): @functools.cached_property def named_inputs(self): named_inputs = self.applied.named_inputs() - del named_inputs[None] + del named_inputs['None'] return named_inputs @abc.abstractmethod From 4d24ed9e6f8b34ba75be475832791ac3dc2f5fe2 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 1 Oct 2022 21:59:01 -0400 Subject: [PATCH 072/173] No default side input. --- sdks/python/apache_beam/runners/dask/dask_runner_test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index 6e69e90938a9..ecf7e920121f 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -58,7 +58,7 @@ def double(x): ])) def test_map_with_side_inputs(self): - def mult_by(x, y=2): + def mult_by(x, y): return x * y with self.pipeline as p: From ee62a4a9a30939c4317c117a2f076b7d8645fdfe Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 1 Oct 2022 22:06:36 -0400 Subject: [PATCH 073/173] Pass along args + kwargs. --- .../runners/dask/dask_runner_test.py | 7 +++++++ .../runners/dask/transform_evaluator.py | 18 +++++------------- 2 files changed, 12 insertions(+), 13 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index ecf7e920121f..aa55dc9beeb9 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -65,6 +65,13 @@ def mult_by(x, y): pcoll = p | beam.Create([1]) | beam.Map(mult_by, 3) assert_that(pcoll, equal_to([3])) + def test_map_with_named_side_inputs(self): + def mult_by(x, y): + return x * y + + with self.pipeline as p: + pcoll = p | beam.Create([1]) | beam.Map(mult_by, y=3) + assert_that(pcoll, equal_to([3])) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 3f4676c8f3e2..d10bdc94f0d1 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -43,13 +43,7 @@ class DaskBagOp(abc.ABC): @property def side_inputs(self): - return self.applied.side_inputs - - @functools.cached_property - def named_inputs(self): - named_inputs = self.applied.named_inputs() - del named_inputs['None'] - return named_inputs + return self.applied.transform.args @abc.abstractmethod def apply(self, input_bag: OpInput) -> db.Bag: @@ -71,16 +65,14 @@ def apply(self, input_bag: OpInput) -> db.Bag: class ParDo(DaskBagOp): def apply(self, input_bag: OpInput) -> db.Bag: - fn = t.cast(apache_beam.ParDo, self.applied.transform).fn - print(self.side_inputs) - print(self.named_inputs) - return input_bag.map(fn.process, *self.side_inputs, **self.named_inputs).flatten() + transform = t.cast(apache_beam.ParDo, self.applied.transform) + return input_bag.map(transform.fn.process, *transform.args, **transform.kwargs).flatten() class Map(DaskBagOp): def apply(self, input_bag: OpInput) -> db.Bag: - fn = t.cast(apache_beam.Map, self.applied.transform).fn - return input_bag.map(fn.process, *self.side_inputs, **self.named_inputs) + transform = t.cast(apache_beam.Map, self.applied.transform) + return input_bag.map(transform.fn.process, *transform.args, **transform.kwargs) class GroupByKey(DaskBagOp): From 506c719eaf59f7f3da012ff6d4ec9b22e163712a Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sun, 9 Oct 2022 08:06:48 -0400 Subject: [PATCH 074/173] Applied yapf to dask sources. --- .../apache_beam/runners/dask/dask_runner.py | 54 +++++++++++-------- .../runners/dask/dask_runner_test.py | 3 ++ .../apache_beam/runners/dask/overrides.py | 24 +++++---- .../runners/dask/transform_evaluator.py | 17 ++++-- 4 files changed, 62 insertions(+), 36 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index 74915c4e254c..ef9a8756caa1 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -50,26 +50,36 @@ def _parse_timeout(candidate): @classmethod def _add_argparse_args(cls, parser: argparse.ArgumentParser) -> None: - parser.add_argument('--dask_client_address', dest='address', type=str, default=None, - help='Address of a dask Scheduler server. Will ' - 'default to a `dask.LocalCluster()`.') - parser.add_argument('--dask_connection_timeout', dest='timeout', - type=DaskOptions._parse_timeout, - help='Timeout duration for initial connection to the ' - 'scheduler.') - parser.add_argument('--dask_scheduler_file', type=str, default=None, - help='Path to a file with scheduler information if ' - 'available.') + parser.add_argument( + '--dask_client_address', + dest='address', + type=str, + default=None, + help='Address of a dask Scheduler server. Will default to a `dask.LocalCluster()`.') + parser.add_argument( + '--dask_connection_timeout', + dest='timeout', + type=DaskOptions._parse_timeout, + help='Timeout duration for initial connection to the scheduler.') + parser.add_argument( + '--dask_scheduler_file', + type=str, + default=None, + help='Path to a file with scheduler information if available.') # TODO(alxr): Add options for security. - parser.add_argument('--dask_client_name', dest='name', type=str, - default=None, - help='Gives the client a name that will be included ' - 'in logs generated on the scheduler for matters ' - 'relating to this client.') - parser.add_argument('--dask_connection_limit', dest='connection_limit', - type=int, default=512, - help='The number of open comms to maintain at once in ' - 'the connection pool.') + parser.add_argument( + '--dask_client_name', + dest='name', + type=str, + default=None, + help='Gives the client a name that will be included in logs generated on the scheduler for matters relating to ' + 'this client.') + parser.add_argument( + '--dask_connection_limit', + dest='connection_limit', + type=int, + default=512, + help='The number of open comms to maintain at once in the connection pool.') @dataclasses.dataclass @@ -115,8 +125,8 @@ def to_dask_bag_visitor() -> PipelineVisitor: @dataclasses.dataclass class DaskBagVisitor(PipelineVisitor): - bags: t.Dict[AppliedPTransform, db.Bag] = dataclasses.field( - default_factory=dict) + bags: t.Dict[AppliedPTransform, + db.Bag] = dataclasses.field(default_factory=dict) def visit_transform(self, transform_node: AppliedPTransform) -> None: op_class = TRANSLATIONS.get(transform_node.transform.__class__, NoOp) @@ -159,7 +169,7 @@ def run_pipeline(self, pipeline, options): 'DaskRunner is not available. Please install apache_beam[dask].') dask_options = options.view_as(DaskOptions).get_all_options( - drop_default=True) + drop_default=True) client = ddist.Client(**dask_options) pipeline.replace_all(dask_overrides()) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index aa55dc9beeb9..e4933eeb11cd 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -40,6 +40,7 @@ def test_create(self): assert_that(pcoll, equal_to([1])) def test_create_and_map(self): + def double(x): return x * 2 @@ -48,6 +49,7 @@ def double(x): assert_that(pcoll, equal_to([2])) def test_create_map_and_groupby(self): + def double(x): return x * 2, x @@ -58,6 +60,7 @@ def double(x): ])) def test_map_with_side_inputs(self): + def mult_by(x, y): return x * y diff --git a/sdks/python/apache_beam/runners/dask/overrides.py b/sdks/python/apache_beam/runners/dask/overrides.py index f06b6de6a10e..0735eba99b03 100644 --- a/sdks/python/apache_beam/runners/dask/overrides.py +++ b/sdks/python/apache_beam/runners/dask/overrides.py @@ -45,6 +45,7 @@ def get_windowing(self, inputs: t.Any) -> beam.Windowing: @typehints.with_input_types(K) @typehints.with_output_types(K) class _Reshuffle(beam.PTransform): + def expand(self, input_or_inputs): return beam.pvalue.PCollection.from_(input_or_inputs) @@ -60,6 +61,7 @@ def expand(self, input_or_inputs): @typehints.with_input_types(t.Tuple[K, V]) @typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) class _GroupByKeyOnly(beam.PTransform): + def expand(self, input_or_inputs): return beam.pvalue.PCollection.from_(input_or_inputs) @@ -89,20 +91,20 @@ def expand(self, input_or_inputs): class _GroupByKey(beam.PTransform): def expand(self, input_or_inputs): - return ( - input_or_inputs - | "GroupByKey" >> _GroupByKeyOnly() - ) + return input_or_inputs | "GroupByKey" >> _GroupByKeyOnly() class _Flatten(beam.PTransform): + def expand(self, input_or_inputs): is_bounded = all(pcoll.is_bounded for pcoll in input_or_inputs) return beam.pvalue.PCollection(self.pipeline, is_bounded=is_bounded) def dask_overrides() -> t.List[PTransformOverride]: + class CreateOverride(PTransformOverride): + def matches(self, applied_ptransform: AppliedPTransform) -> bool: return applied_ptransform.transform.__class__ == beam.Create @@ -111,6 +113,7 @@ def get_replacement_transform_for_applied_ptransform( return _Create(t.cast(beam.Create, applied_ptransform.transform).values) class ReshuffleOverride(PTransformOverride): + def matches(self, applied_ptransform: AppliedPTransform) -> bool: return applied_ptransform.transform.__class__ == beam.Reshuffle @@ -119,6 +122,7 @@ def get_replacement_transform_for_applied_ptransform( return _Reshuffle() class ReadOverride(PTransformOverride): + def matches(self, applied_ptransform: AppliedPTransform) -> bool: return applied_ptransform.transform.__class__ == beam.io.Read @@ -127,6 +131,7 @@ def get_replacement_transform_for_applied_ptransform( return _Read(t.cast(beam.io.Read, applied_ptransform.transform).source) class GroupByKeyOverride(PTransformOverride): + def matches(self, applied_ptransform: AppliedPTransform) -> bool: return applied_ptransform.transform.__class__ == beam.GroupByKey @@ -135,6 +140,7 @@ def get_replacement_transform_for_applied_ptransform( return _GroupByKey() class FlattenOverride(PTransformOverride): + def matches(self, applied_ptransform: AppliedPTransform) -> bool: return applied_ptransform.transform.__class__ == beam.Flatten @@ -143,9 +149,9 @@ def get_replacement_transform_for_applied_ptransform( return _Flatten() return [ - CreateOverride(), - ReshuffleOverride(), - ReadOverride(), - GroupByKeyOverride(), - FlattenOverride(), + CreateOverride(), + ReshuffleOverride(), + ReadOverride(), + GroupByKeyOverride(), + FlattenOverride(), ] diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index d10bdc94f0d1..bc3516828249 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -14,6 +14,7 @@ # See the License for the specific language governing permissions and # limitations under the License. # + """Transform Beam PTransforms into Dask Bag operations. A minimum set of operation substitutions, to adap Beam's PTransform model @@ -51,11 +52,13 @@ def apply(self, input_bag: OpInput) -> db.Bag: class NoOp(DaskBagOp): + def apply(self, input_bag: OpInput) -> db.Bag: return input_bag class Create(DaskBagOp): + def apply(self, input_bag: OpInput) -> db.Bag: assert input_bag is None, 'Create expects no input!' original_transform = t.cast(_Create, self.applied.transform) @@ -64,18 +67,21 @@ def apply(self, input_bag: OpInput) -> db.Bag: class ParDo(DaskBagOp): + def apply(self, input_bag: OpInput) -> db.Bag: transform = t.cast(apache_beam.ParDo, self.applied.transform) return input_bag.map(transform.fn.process, *transform.args, **transform.kwargs).flatten() class Map(DaskBagOp): + def apply(self, input_bag: OpInput) -> db.Bag: transform = t.cast(apache_beam.Map, self.applied.transform) return input_bag.map(transform.fn.process, *transform.args, **transform.kwargs) class GroupByKey(DaskBagOp): + def apply(self, input_bag: OpInput) -> db.Bag: def key(item): return item[0] @@ -88,15 +94,16 @@ def value(item): class Flatten(DaskBagOp): + def apply(self, input_bag: OpInput) -> db.Bag: assert type(input_bag) is list, 'Must take a sequence of bags!' return db.concat(input_bag) TRANSLATIONS = { - _Create: Create, - apache_beam.ParDo: ParDo, - apache_beam.Map: Map, - _GroupByKeyOnly: GroupByKey, - _Flatten: Flatten, + _Create: Create, + apache_beam.ParDo: ParDo, + apache_beam.Map: Map, + _GroupByKeyOnly: GroupByKey, + _Flatten: Flatten, } From cd0ba8b70f20463e14cb35ae28dc66236cd71060 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sun, 9 Oct 2022 08:12:50 -0400 Subject: [PATCH 075/173] Dask sources passing pylint. --- .../apache_beam/runners/dask/dask_runner.py | 10 ++++++---- .../runners/dask/transform_evaluator.py | 17 ++++++++++++----- 2 files changed, 18 insertions(+), 9 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index ef9a8756caa1..ed43c4fb5a72 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -55,7 +55,8 @@ def _add_argparse_args(cls, parser: argparse.ArgumentParser) -> None: dest='address', type=str, default=None, - help='Address of a dask Scheduler server. Will default to a `dask.LocalCluster()`.') + help='Address of a dask Scheduler server. Will default to a ' + '`dask.LocalCluster()`.') parser.add_argument( '--dask_connection_timeout', dest='timeout', @@ -72,14 +73,15 @@ def _add_argparse_args(cls, parser: argparse.ArgumentParser) -> None: dest='name', type=str, default=None, - help='Gives the client a name that will be included in logs generated on the scheduler for matters relating to ' - 'this client.') + help='Gives the client a name that will be included in logs generated on ' + 'the scheduler for matters relating to this client.') parser.add_argument( '--dask_connection_limit', dest='connection_limit', type=int, default=512, - help='The number of open comms to maintain at once in the connection pool.') + help='The number of open comms to maintain at once in the connection ' + 'pool.') @dataclasses.dataclass diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index bc3516828249..e6fb5c336554 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -22,12 +22,11 @@ TODO(alxr): Translate ops from https://docs.dask.org/en/latest/bag-api.html. """ +import abc import dataclasses +import typing as t -import abc import dask.bag as db -import typing as t -import functools import apache_beam from apache_beam.pipeline import AppliedPTransform @@ -70,14 +69,22 @@ class ParDo(DaskBagOp): def apply(self, input_bag: OpInput) -> db.Bag: transform = t.cast(apache_beam.ParDo, self.applied.transform) - return input_bag.map(transform.fn.process, *transform.args, **transform.kwargs).flatten() + return input_bag.map( + transform.fn.process, + *transform.args, + **transform.kwargs + ).flatten() class Map(DaskBagOp): def apply(self, input_bag: OpInput) -> db.Bag: transform = t.cast(apache_beam.Map, self.applied.transform) - return input_bag.map(transform.fn.process, *transform.args, **transform.kwargs) + return input_bag.map( + transform.fn.process, + *transform.args, + **transform.kwargs + ) class GroupByKey(DaskBagOp): From d0a7c63fd52ef746c2f340e26ae8c791c6f6a07d Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sun, 9 Oct 2022 08:16:19 -0400 Subject: [PATCH 076/173] Added dask extra to docs gen tox env. --- sdks/python/tox.ini | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini index 95ca964ae00f..0d25f5c9d948 100644 --- a/sdks/python/tox.ini +++ b/sdks/python/tox.ini @@ -141,7 +141,7 @@ commands = python setup.py mypy [testenv:py38-docs] -extras = test,gcp,docs,interactive,dataframe +extras = test,gcp,docs,interactive,dataframe,dask deps = Sphinx==1.8.5 sphinx_rtd_theme==0.4.3 From 775bd077b043e108239602eabb7c7701c29b3b06 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sun, 9 Oct 2022 08:23:25 -0400 Subject: [PATCH 077/173] Applied yapf from tox. --- .../apache_beam/runners/dask/dask_runner.py | 57 +++++++++---------- .../runners/dask/dask_runner_test.py | 8 +-- .../apache_beam/runners/dask/overrides.py | 11 ---- .../runners/dask/transform_evaluator.py | 16 +----- 4 files changed, 31 insertions(+), 61 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index ed43c4fb5a72..60b9e43ed10e 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -14,6 +14,7 @@ # See the License for the specific language governing permissions and # limitations under the License. # + """DaskRunner, executing remote jobs on Dask.distributed. The DaskRunner is a runner implementation that executes a graph of @@ -39,7 +40,6 @@ class DaskOptions(PipelineOptions): - @staticmethod def _parse_timeout(candidate): try: @@ -51,37 +51,37 @@ def _parse_timeout(candidate): @classmethod def _add_argparse_args(cls, parser: argparse.ArgumentParser) -> None: parser.add_argument( - '--dask_client_address', - dest='address', - type=str, - default=None, - help='Address of a dask Scheduler server. Will default to a ' - '`dask.LocalCluster()`.') + '--dask_client_address', + dest='address', + type=str, + default=None, + help='Address of a dask Scheduler server. Will default to a ' + '`dask.LocalCluster()`.') parser.add_argument( - '--dask_connection_timeout', - dest='timeout', - type=DaskOptions._parse_timeout, - help='Timeout duration for initial connection to the scheduler.') + '--dask_connection_timeout', + dest='timeout', + type=DaskOptions._parse_timeout, + help='Timeout duration for initial connection to the scheduler.') parser.add_argument( - '--dask_scheduler_file', - type=str, - default=None, - help='Path to a file with scheduler information if available.') + '--dask_scheduler_file', + type=str, + default=None, + help='Path to a file with scheduler information if available.') # TODO(alxr): Add options for security. parser.add_argument( - '--dask_client_name', - dest='name', - type=str, - default=None, - help='Gives the client a name that will be included in logs generated on ' - 'the scheduler for matters relating to this client.') + '--dask_client_name', + dest='name', + type=str, + default=None, + help='Gives the client a name that will be included in logs generated on ' + 'the scheduler for matters relating to this client.') parser.add_argument( - '--dask_connection_limit', - dest='connection_limit', - type=int, - default=512, - help='The number of open comms to maintain at once in the connection ' - 'pool.') + '--dask_connection_limit', + dest='connection_limit', + type=int, + default=512, + help='The number of open comms to maintain at once in the connection ' + 'pool.') @dataclasses.dataclass @@ -120,7 +120,6 @@ def metrics(self): class DaskRunner(BundleBasedDirectRunner): """Executes a pipeline on a Dask distributed client.""" - @staticmethod def to_dask_bag_visitor() -> PipelineVisitor: from dask import bag as db @@ -168,7 +167,7 @@ def run_pipeline(self, pipeline, options): import dask.distributed as ddist except ImportError: raise ImportError( - 'DaskRunner is not available. Please install apache_beam[dask].') + 'DaskRunner is not available. Please install apache_beam[dask].') dask_options = options.view_as(DaskOptions).get_all_options( drop_default=True) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index e4933eeb11cd..a75b1f2fb94a 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -30,7 +30,6 @@ class DaskRunnerRunPipelineTest(unittest.TestCase): """Test class used to introspect the dask runner via a debugger.""" - def setUp(self) -> None: self.pipeline = test_pipeline.TestPipeline(runner=DaskRunner()) @@ -40,7 +39,6 @@ def test_create(self): assert_that(pcoll, equal_to([1])) def test_create_and_map(self): - def double(x): return x * 2 @@ -49,18 +47,14 @@ def double(x): assert_that(pcoll, equal_to([2])) def test_create_map_and_groupby(self): - def double(x): return x * 2, x with self.pipeline as p: pcoll = p | beam.Create([1]) | beam.Map(double) | beam.GroupByKey() - assert_that(pcoll, equal_to([ - (2, [1]) - ])) + assert_that(pcoll, equal_to([(2, [1])])) def test_map_with_side_inputs(self): - def mult_by(x, y): return x * y diff --git a/sdks/python/apache_beam/runners/dask/overrides.py b/sdks/python/apache_beam/runners/dask/overrides.py index 0735eba99b03..7528e0132d5a 100644 --- a/sdks/python/apache_beam/runners/dask/overrides.py +++ b/sdks/python/apache_beam/runners/dask/overrides.py @@ -45,7 +45,6 @@ def get_windowing(self, inputs: t.Any) -> beam.Windowing: @typehints.with_input_types(K) @typehints.with_output_types(K) class _Reshuffle(beam.PTransform): - def expand(self, input_or_inputs): return beam.pvalue.PCollection.from_(input_or_inputs) @@ -61,7 +60,6 @@ def expand(self, input_or_inputs): @typehints.with_input_types(t.Tuple[K, V]) @typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) class _GroupByKeyOnly(beam.PTransform): - def expand(self, input_or_inputs): return beam.pvalue.PCollection.from_(input_or_inputs) @@ -77,7 +75,6 @@ def infer_output_type(self, input_type): @typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) class _GroupAlsoByWindow(beam.ParDo): """Not used yet...""" - def __init__(self, windowing): super().__init__(_GroupAlsoByWindowDoFn(windowing)) self.windowing = windowing @@ -89,22 +86,18 @@ def expand(self, input_or_inputs): @typehints.with_input_types(t.Tuple[K, V]) @typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) class _GroupByKey(beam.PTransform): - def expand(self, input_or_inputs): return input_or_inputs | "GroupByKey" >> _GroupByKeyOnly() class _Flatten(beam.PTransform): - def expand(self, input_or_inputs): is_bounded = all(pcoll.is_bounded for pcoll in input_or_inputs) return beam.pvalue.PCollection(self.pipeline, is_bounded=is_bounded) def dask_overrides() -> t.List[PTransformOverride]: - class CreateOverride(PTransformOverride): - def matches(self, applied_ptransform: AppliedPTransform) -> bool: return applied_ptransform.transform.__class__ == beam.Create @@ -113,7 +106,6 @@ def get_replacement_transform_for_applied_ptransform( return _Create(t.cast(beam.Create, applied_ptransform.transform).values) class ReshuffleOverride(PTransformOverride): - def matches(self, applied_ptransform: AppliedPTransform) -> bool: return applied_ptransform.transform.__class__ == beam.Reshuffle @@ -122,7 +114,6 @@ def get_replacement_transform_for_applied_ptransform( return _Reshuffle() class ReadOverride(PTransformOverride): - def matches(self, applied_ptransform: AppliedPTransform) -> bool: return applied_ptransform.transform.__class__ == beam.io.Read @@ -131,7 +122,6 @@ def get_replacement_transform_for_applied_ptransform( return _Read(t.cast(beam.io.Read, applied_ptransform.transform).source) class GroupByKeyOverride(PTransformOverride): - def matches(self, applied_ptransform: AppliedPTransform) -> bool: return applied_ptransform.transform.__class__ == beam.GroupByKey @@ -140,7 +130,6 @@ def get_replacement_transform_for_applied_ptransform( return _GroupByKey() class FlattenOverride(PTransformOverride): - def matches(self, applied_ptransform: AppliedPTransform) -> bool: return applied_ptransform.transform.__class__ == beam.Flatten diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index e6fb5c336554..7ff31ef505ce 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -51,13 +51,11 @@ def apply(self, input_bag: OpInput) -> db.Bag: class NoOp(DaskBagOp): - def apply(self, input_bag: OpInput) -> db.Bag: return input_bag class Create(DaskBagOp): - def apply(self, input_bag: OpInput) -> db.Bag: assert input_bag is None, 'Create expects no input!' original_transform = t.cast(_Create, self.applied.transform) @@ -66,29 +64,20 @@ def apply(self, input_bag: OpInput) -> db.Bag: class ParDo(DaskBagOp): - def apply(self, input_bag: OpInput) -> db.Bag: transform = t.cast(apache_beam.ParDo, self.applied.transform) return input_bag.map( - transform.fn.process, - *transform.args, - **transform.kwargs - ).flatten() + transform.fn.process, *transform.args, **transform.kwargs).flatten() class Map(DaskBagOp): - def apply(self, input_bag: OpInput) -> db.Bag: transform = t.cast(apache_beam.Map, self.applied.transform) return input_bag.map( - transform.fn.process, - *transform.args, - **transform.kwargs - ) + transform.fn.process, *transform.args, **transform.kwargs) class GroupByKey(DaskBagOp): - def apply(self, input_bag: OpInput) -> db.Bag: def key(item): return item[0] @@ -101,7 +90,6 @@ def value(item): class Flatten(DaskBagOp): - def apply(self, input_bag: OpInput) -> db.Bag: assert type(input_bag) is list, 'Must take a sequence of bags!' return db.concat(input_bag) From efba1c9863cb56b6843fd02f7b0c337569e8d3c1 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sun, 9 Oct 2022 09:54:04 -0400 Subject: [PATCH 078/173] Include dask in mypy checks. --- sdks/python/tox.ini | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini index 0d25f5c9d948..dbb51480718f 100644 --- a/sdks/python/tox.ini +++ b/sdks/python/tox.ini @@ -135,7 +135,7 @@ deps = mypy==0.782 # make extras available in case any of these libs are typed extras = - gcp + gcp,dask commands = mypy --version python setup.py mypy From 741d96175812b6907ec9004c31452cd9ac5b7b08 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sun, 9 Oct 2022 10:01:32 -0400 Subject: [PATCH 079/173] Upgrading mypy support to python 3.8 since py37 support is deprecated in dask. --- sdks/python/tox.ini | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini index dbb51480718f..56bb080fb9c3 100644 --- a/sdks/python/tox.ini +++ b/sdks/python/tox.ini @@ -129,7 +129,7 @@ deps = commands = time {toxinidir}/scripts/run_whitespacelint.sh -[testenv:py37-mypy] +[testenv:py38-mypy] deps = -r build-requirements.txt mypy==0.782 @@ -140,6 +140,7 @@ commands = mypy --version python setup.py mypy + [testenv:py38-docs] extras = test,gcp,docs,interactive,dataframe,dask deps = From f66458ad2a7d8792b933d6a4ee8bfc5ac26ef945 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sun, 9 Oct 2022 10:06:28 -0400 Subject: [PATCH 080/173] Manually installing an old version of dask before 3.7 support was dropped. --- sdks/python/tox.ini | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini index 56bb080fb9c3..7e92ce28f85f 100644 --- a/sdks/python/tox.ini +++ b/sdks/python/tox.ini @@ -129,13 +129,15 @@ deps = commands = time {toxinidir}/scripts/run_whitespacelint.sh -[testenv:py38-mypy] +[testenv:py37-mypy] deps = -r build-requirements.txt mypy==0.782 + dask==2022.01.0 + distributed==2022.01.0 # make extras available in case any of these libs are typed extras = - gcp,dask + gcp commands = mypy --version python setup.py mypy From 5dcf9698eab7f398f7038bd55b23759b7d88b21c Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sun, 9 Oct 2022 10:13:28 -0400 Subject: [PATCH 081/173] fix lint: line too long. --- sdks/python/apache_beam/runners/dask/dask_runner.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index 60b9e43ed10e..15a475339ab8 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -73,8 +73,8 @@ def _add_argparse_args(cls, parser: argparse.ArgumentParser) -> None: dest='name', type=str, default=None, - help='Gives the client a name that will be included in logs generated on ' - 'the scheduler for matters relating to this client.') + help='Gives the client a name that will be included in logs generated ' + 'on the scheduler for matters relating to this client.') parser.add_argument( '--dask_connection_limit', dest='connection_limit', From ec5f613180cb064048e06d685ba3fba878d58d6d Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sun, 9 Oct 2022 10:21:31 -0400 Subject: [PATCH 082/173] Fixed type errors with DaskRunnerResult. Disabled mypy type checking in dask. --- sdks/python/apache_beam/runners/dask/dask_runner.py | 4 ++-- sdks/python/mypy.ini | 3 +++ 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index 15a475339ab8..afce1edac52f 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -94,7 +94,7 @@ class DaskRunnerResult(PipelineResult): def __post_init__(self): super().__init__(PipelineState.RUNNING) - def wait_until_finish(self, duration=None) -> PipelineState: + def wait_until_finish(self, duration=None) -> str: try: if duration is not None: # Convert milliseconds to seconds @@ -107,7 +107,7 @@ def wait_until_finish(self, duration=None) -> PipelineState: raise return self._state - def cancel(self) -> PipelineState: + def cancel(self) -> str: self._state = PipelineState.CANCELLING self.client.cancel(self.futures) self._state = PipelineState.CANCELLED diff --git a/sdks/python/mypy.ini b/sdks/python/mypy.ini index 9309120a8cab..a628036d6682 100644 --- a/sdks/python/mypy.ini +++ b/sdks/python/mypy.ini @@ -89,6 +89,9 @@ ignore_errors = true [mypy-apache_beam.runners.direct.*] ignore_errors = true +[mypy-apache_beam.runners.dask.*] +ignore_errors = true + [mypy-apache_beam.runners.interactive.*] ignore_errors = true From 04b1f1af032bd0839045a24f5d94003e05c2fa22 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sun, 9 Oct 2022 10:31:39 -0400 Subject: [PATCH 083/173] Fix pytype errors (in transform_evaluator). --- sdks/python/apache_beam/runners/dask/transform_evaluator.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 7ff31ef505ce..0f10f6d8211d 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -64,21 +64,21 @@ def apply(self, input_bag: OpInput) -> db.Bag: class ParDo(DaskBagOp): - def apply(self, input_bag: OpInput) -> db.Bag: + def apply(self, input_bag: db.Bag) -> db.Bag: transform = t.cast(apache_beam.ParDo, self.applied.transform) return input_bag.map( transform.fn.process, *transform.args, **transform.kwargs).flatten() class Map(DaskBagOp): - def apply(self, input_bag: OpInput) -> db.Bag: + def apply(self, input_bag: db.Bag) -> db.Bag: transform = t.cast(apache_beam.Map, self.applied.transform) return input_bag.map( transform.fn.process, *transform.args, **transform.kwargs) class GroupByKey(DaskBagOp): - def apply(self, input_bag: OpInput) -> db.Bag: + def apply(self, input_bag: db.Bag) -> db.Bag: def key(item): return item[0] From 712944bd1327f3d85c2f9150f56c9e57d21c7b8a Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sun, 9 Oct 2022 10:36:08 -0400 Subject: [PATCH 084/173] Ran isort. --- sdks/python/apache_beam/runners/dask/dask_runner.py | 12 ++++-------- .../apache_beam/runners/dask/dask_runner_test.py | 10 +++++----- sdks/python/apache_beam/runners/dask/overrides.py | 4 +--- .../apache_beam/runners/dask/transform_evaluator.py | 5 ++--- 4 files changed, 12 insertions(+), 19 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index afce1edac52f..b3c2b9eacfb3 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -21,21 +21,17 @@ transformations across processes and workers via Dask distributed's scheduler. """ -import dataclasses - import argparse +import dataclasses import typing as t from apache_beam import pvalue from apache_beam.options.pipeline_options import PipelineOptions -from apache_beam.pipeline import AppliedPTransform -from apache_beam.pipeline import PipelineVisitor +from apache_beam.pipeline import AppliedPTransform, PipelineVisitor from apache_beam.runners.dask.overrides import dask_overrides -from apache_beam.runners.dask.transform_evaluator import TRANSLATIONS -from apache_beam.runners.dask.transform_evaluator import NoOp +from apache_beam.runners.dask.transform_evaluator import TRANSLATIONS, NoOp from apache_beam.runners.direct.direct_runner import BundleBasedDirectRunner -from apache_beam.runners.runner import PipelineResult -from apache_beam.runners.runner import PipelineState +from apache_beam.runners.runner import PipelineResult, PipelineState from apache_beam.utils.interactive_utils import is_in_notebook diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index a75b1f2fb94a..972524289c86 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -16,16 +16,16 @@ # import unittest +import apache_beam as beam +from apache_beam.runners.dask.dask_runner import DaskRunner +from apache_beam.testing import test_pipeline +from apache_beam.testing.util import assert_that, equal_to + try: import dask # pylint: disable=unused-import except (ImportError, ModuleNotFoundError): raise unittest.SkipTest('Dask must be installed to run tests.') -import apache_beam as beam -from apache_beam.runners.dask.dask_runner import DaskRunner -from apache_beam.testing import test_pipeline -from apache_beam.testing.util import assert_that -from apache_beam.testing.util import equal_to class DaskRunnerRunPipelineTest(unittest.TestCase): diff --git a/sdks/python/apache_beam/runners/dask/overrides.py b/sdks/python/apache_beam/runners/dask/overrides.py index 7528e0132d5a..3a4e7b7345c7 100644 --- a/sdks/python/apache_beam/runners/dask/overrides.py +++ b/sdks/python/apache_beam/runners/dask/overrides.py @@ -15,14 +15,12 @@ # limitations under the License. # import dataclasses - import typing as t import apache_beam as beam from apache_beam import typehints from apache_beam.io.iobase import SourceBase -from apache_beam.pipeline import AppliedPTransform -from apache_beam.pipeline import PTransformOverride +from apache_beam.pipeline import AppliedPTransform, PTransformOverride from apache_beam.runners.direct.direct_runner import _GroupAlsoByWindowDoFn from apache_beam.transforms import ptransform from apache_beam.transforms.window import GlobalWindows diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 0f10f6d8211d..80fca45a1172 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -30,9 +30,8 @@ import apache_beam from apache_beam.pipeline import AppliedPTransform -from apache_beam.runners.dask.overrides import _Create -from apache_beam.runners.dask.overrides import _Flatten -from apache_beam.runners.dask.overrides import _GroupByKeyOnly +from apache_beam.runners.dask.overrides import (_Create, _Flatten, + _GroupByKeyOnly) OpInput = t.Union[db.Bag, t.Sequence[db.Bag], None] From 567b72b4fcbb7204f2a49fc4cbd2f33dc4908778 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sun, 9 Oct 2022 10:47:26 -0400 Subject: [PATCH 085/173] Ran yapf again. --- sdks/python/apache_beam/runners/dask/dask_runner.py | 2 ++ .../apache_beam/runners/dask/dask_runner_test.py | 6 +++++- sdks/python/apache_beam/runners/dask/overrides.py | 11 +++++++++++ .../apache_beam/runners/dask/transform_evaluator.py | 12 ++++++++++-- 4 files changed, 28 insertions(+), 3 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index b3c2b9eacfb3..e1303039848e 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -36,6 +36,7 @@ class DaskOptions(PipelineOptions): + @staticmethod def _parse_timeout(candidate): try: @@ -116,6 +117,7 @@ def metrics(self): class DaskRunner(BundleBasedDirectRunner): """Executes a pipeline on a Dask distributed client.""" + @staticmethod def to_dask_bag_visitor() -> PipelineVisitor: from dask import bag as db diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index 972524289c86..58fa7b7ac439 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -27,9 +27,9 @@ raise unittest.SkipTest('Dask must be installed to run tests.') - class DaskRunnerRunPipelineTest(unittest.TestCase): """Test class used to introspect the dask runner via a debugger.""" + def setUp(self) -> None: self.pipeline = test_pipeline.TestPipeline(runner=DaskRunner()) @@ -39,6 +39,7 @@ def test_create(self): assert_that(pcoll, equal_to([1])) def test_create_and_map(self): + def double(x): return x * 2 @@ -47,6 +48,7 @@ def double(x): assert_that(pcoll, equal_to([2])) def test_create_map_and_groupby(self): + def double(x): return x * 2, x @@ -55,6 +57,7 @@ def double(x): assert_that(pcoll, equal_to([(2, [1])])) def test_map_with_side_inputs(self): + def mult_by(x, y): return x * y @@ -63,6 +66,7 @@ def mult_by(x, y): assert_that(pcoll, equal_to([3])) def test_map_with_named_side_inputs(self): + def mult_by(x, y): return x * y diff --git a/sdks/python/apache_beam/runners/dask/overrides.py b/sdks/python/apache_beam/runners/dask/overrides.py index 3a4e7b7345c7..4854230b22d2 100644 --- a/sdks/python/apache_beam/runners/dask/overrides.py +++ b/sdks/python/apache_beam/runners/dask/overrides.py @@ -43,6 +43,7 @@ def get_windowing(self, inputs: t.Any) -> beam.Windowing: @typehints.with_input_types(K) @typehints.with_output_types(K) class _Reshuffle(beam.PTransform): + def expand(self, input_or_inputs): return beam.pvalue.PCollection.from_(input_or_inputs) @@ -58,6 +59,7 @@ def expand(self, input_or_inputs): @typehints.with_input_types(t.Tuple[K, V]) @typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) class _GroupByKeyOnly(beam.PTransform): + def expand(self, input_or_inputs): return beam.pvalue.PCollection.from_(input_or_inputs) @@ -73,6 +75,7 @@ def infer_output_type(self, input_type): @typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) class _GroupAlsoByWindow(beam.ParDo): """Not used yet...""" + def __init__(self, windowing): super().__init__(_GroupAlsoByWindowDoFn(windowing)) self.windowing = windowing @@ -84,18 +87,22 @@ def expand(self, input_or_inputs): @typehints.with_input_types(t.Tuple[K, V]) @typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) class _GroupByKey(beam.PTransform): + def expand(self, input_or_inputs): return input_or_inputs | "GroupByKey" >> _GroupByKeyOnly() class _Flatten(beam.PTransform): + def expand(self, input_or_inputs): is_bounded = all(pcoll.is_bounded for pcoll in input_or_inputs) return beam.pvalue.PCollection(self.pipeline, is_bounded=is_bounded) def dask_overrides() -> t.List[PTransformOverride]: + class CreateOverride(PTransformOverride): + def matches(self, applied_ptransform: AppliedPTransform) -> bool: return applied_ptransform.transform.__class__ == beam.Create @@ -104,6 +111,7 @@ def get_replacement_transform_for_applied_ptransform( return _Create(t.cast(beam.Create, applied_ptransform.transform).values) class ReshuffleOverride(PTransformOverride): + def matches(self, applied_ptransform: AppliedPTransform) -> bool: return applied_ptransform.transform.__class__ == beam.Reshuffle @@ -112,6 +120,7 @@ def get_replacement_transform_for_applied_ptransform( return _Reshuffle() class ReadOverride(PTransformOverride): + def matches(self, applied_ptransform: AppliedPTransform) -> bool: return applied_ptransform.transform.__class__ == beam.io.Read @@ -120,6 +129,7 @@ def get_replacement_transform_for_applied_ptransform( return _Read(t.cast(beam.io.Read, applied_ptransform.transform).source) class GroupByKeyOverride(PTransformOverride): + def matches(self, applied_ptransform: AppliedPTransform) -> bool: return applied_ptransform.transform.__class__ == beam.GroupByKey @@ -128,6 +138,7 @@ def get_replacement_transform_for_applied_ptransform( return _GroupByKey() class FlattenOverride(PTransformOverride): + def matches(self, applied_ptransform: AppliedPTransform) -> bool: return applied_ptransform.transform.__class__ == beam.Flatten diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 80fca45a1172..12e38d90adde 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -30,8 +30,9 @@ import apache_beam from apache_beam.pipeline import AppliedPTransform -from apache_beam.runners.dask.overrides import (_Create, _Flatten, - _GroupByKeyOnly) +from apache_beam.runners.dask.overrides import ( + _Create, _Flatten, _GroupByKeyOnly +) OpInput = t.Union[db.Bag, t.Sequence[db.Bag], None] @@ -50,11 +51,13 @@ def apply(self, input_bag: OpInput) -> db.Bag: class NoOp(DaskBagOp): + def apply(self, input_bag: OpInput) -> db.Bag: return input_bag class Create(DaskBagOp): + def apply(self, input_bag: OpInput) -> db.Bag: assert input_bag is None, 'Create expects no input!' original_transform = t.cast(_Create, self.applied.transform) @@ -63,6 +66,7 @@ def apply(self, input_bag: OpInput) -> db.Bag: class ParDo(DaskBagOp): + def apply(self, input_bag: db.Bag) -> db.Bag: transform = t.cast(apache_beam.ParDo, self.applied.transform) return input_bag.map( @@ -70,6 +74,7 @@ def apply(self, input_bag: db.Bag) -> db.Bag: class Map(DaskBagOp): + def apply(self, input_bag: db.Bag) -> db.Bag: transform = t.cast(apache_beam.Map, self.applied.transform) return input_bag.map( @@ -77,7 +82,9 @@ def apply(self, input_bag: db.Bag) -> db.Bag: class GroupByKey(DaskBagOp): + def apply(self, input_bag: db.Bag) -> db.Bag: + def key(item): return item[0] @@ -89,6 +96,7 @@ def value(item): class Flatten(DaskBagOp): + def apply(self, input_bag: OpInput) -> db.Bag: assert type(input_bag) is list, 'Must take a sequence of bags!' return db.concat(input_bag) From f53c0a47a14073c208f06243531b61fcdc6461eb Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Mon, 10 Oct 2022 14:53:39 -0400 Subject: [PATCH 086/173] Fix imports (one per line) --- sdks/python/apache_beam/runners/dask/dask_runner.py | 9 ++++++--- sdks/python/apache_beam/runners/dask/dask_runner_test.py | 3 ++- sdks/python/apache_beam/runners/dask/overrides.py | 3 ++- .../apache_beam/runners/dask/transform_evaluator.py | 6 +++--- 4 files changed, 13 insertions(+), 8 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index e1303039848e..78af1c2c3e8b 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -27,11 +27,14 @@ from apache_beam import pvalue from apache_beam.options.pipeline_options import PipelineOptions -from apache_beam.pipeline import AppliedPTransform, PipelineVisitor +from apache_beam.pipeline import AppliedPTransform +from apache_beam.pipeline import PipelineVisitor from apache_beam.runners.dask.overrides import dask_overrides -from apache_beam.runners.dask.transform_evaluator import TRANSLATIONS, NoOp +from apache_beam.runners.dask.transform_evaluator import NoOp +from apache_beam.runners.dask.transform_evaluator import TRANSLATIONS from apache_beam.runners.direct.direct_runner import BundleBasedDirectRunner -from apache_beam.runners.runner import PipelineResult, PipelineState +from apache_beam.runners.runner import PipelineResult +from apache_beam.runners.runner import PipelineState from apache_beam.utils.interactive_utils import is_in_notebook diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index 58fa7b7ac439..02f4213c9be6 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -19,7 +19,8 @@ import apache_beam as beam from apache_beam.runners.dask.dask_runner import DaskRunner from apache_beam.testing import test_pipeline -from apache_beam.testing.util import assert_that, equal_to +from apache_beam.testing.util import assert_that +from apache_beam.testing.util import equal_to try: import dask # pylint: disable=unused-import diff --git a/sdks/python/apache_beam/runners/dask/overrides.py b/sdks/python/apache_beam/runners/dask/overrides.py index 4854230b22d2..7bcfb1d19a79 100644 --- a/sdks/python/apache_beam/runners/dask/overrides.py +++ b/sdks/python/apache_beam/runners/dask/overrides.py @@ -20,7 +20,8 @@ import apache_beam as beam from apache_beam import typehints from apache_beam.io.iobase import SourceBase -from apache_beam.pipeline import AppliedPTransform, PTransformOverride +from apache_beam.pipeline import AppliedPTransform +from apache_beam.pipeline import PTransformOverride from apache_beam.runners.direct.direct_runner import _GroupAlsoByWindowDoFn from apache_beam.transforms import ptransform from apache_beam.transforms.window import GlobalWindows diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 12e38d90adde..090350275bcf 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -30,9 +30,9 @@ import apache_beam from apache_beam.pipeline import AppliedPTransform -from apache_beam.runners.dask.overrides import ( - _Create, _Flatten, _GroupByKeyOnly -) +from apache_beam.runners.dask.overrides import _Create +from apache_beam.runners.dask.overrides import _Flatten +from apache_beam.runners.dask.overrides import _GroupByKeyOnly OpInput = t.Union[db.Bag, t.Sequence[db.Bag], None] From fb280ad976ae3dd8f37225e3822352288d6cd471 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Mon, 10 Oct 2022 15:04:19 -0400 Subject: [PATCH 087/173] isort -- alphabetical. --- sdks/python/apache_beam/runners/dask/dask_runner.py | 2 +- sdks/python/apache_beam/runners/dask/transform_evaluator.py | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index 78af1c2c3e8b..cc6d89a04866 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -30,8 +30,8 @@ from apache_beam.pipeline import AppliedPTransform from apache_beam.pipeline import PipelineVisitor from apache_beam.runners.dask.overrides import dask_overrides -from apache_beam.runners.dask.transform_evaluator import NoOp from apache_beam.runners.dask.transform_evaluator import TRANSLATIONS +from apache_beam.runners.dask.transform_evaluator import NoOp from apache_beam.runners.direct.direct_runner import BundleBasedDirectRunner from apache_beam.runners.runner import PipelineResult from apache_beam.runners.runner import PipelineState diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 090350275bcf..65635952bdbc 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -26,9 +26,8 @@ import dataclasses import typing as t -import dask.bag as db - import apache_beam +import dask.bag as db from apache_beam.pipeline import AppliedPTransform from apache_beam.runners.dask.overrides import _Create from apache_beam.runners.dask.overrides import _Flatten From 80ddfecd37653e0e3fcc47386ca08c3f85f0005e Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Mon, 10 Oct 2022 15:55:15 -0400 Subject: [PATCH 088/173] Added feature to CHANGES.md. --- CHANGES.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGES.md b/CHANGES.md index 92b578d16c85..35048f359fd3 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -67,6 +67,7 @@ ## New Features / Improvements * X feature added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). +* Dask support for the Python SDK (Python) ([#18962](https://github.com/apache/beam/issues/18962)). ## Breaking Changes From 40c4e35c0d28adad9a3189bc0955f0db2f78738a Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Mon, 10 Oct 2022 20:51:46 +0000 Subject: [PATCH 089/173] ran yapf via tox on linux machine --- sdks/python/apache_beam/runners/dask/dask_runner.py | 2 -- .../apache_beam/runners/dask/dask_runner_test.py | 5 ----- sdks/python/apache_beam/runners/dask/overrides.py | 11 ----------- .../apache_beam/runners/dask/transform_evaluator.py | 7 ------- 4 files changed, 25 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index cc6d89a04866..e0bd71f8c9f6 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -39,7 +39,6 @@ class DaskOptions(PipelineOptions): - @staticmethod def _parse_timeout(candidate): try: @@ -120,7 +119,6 @@ def metrics(self): class DaskRunner(BundleBasedDirectRunner): """Executes a pipeline on a Dask distributed client.""" - @staticmethod def to_dask_bag_visitor() -> PipelineVisitor: from dask import bag as db diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index 02f4213c9be6..f6a59e0dc3fc 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -30,7 +30,6 @@ class DaskRunnerRunPipelineTest(unittest.TestCase): """Test class used to introspect the dask runner via a debugger.""" - def setUp(self) -> None: self.pipeline = test_pipeline.TestPipeline(runner=DaskRunner()) @@ -40,7 +39,6 @@ def test_create(self): assert_that(pcoll, equal_to([1])) def test_create_and_map(self): - def double(x): return x * 2 @@ -49,7 +47,6 @@ def double(x): assert_that(pcoll, equal_to([2])) def test_create_map_and_groupby(self): - def double(x): return x * 2, x @@ -58,7 +55,6 @@ def double(x): assert_that(pcoll, equal_to([(2, [1])])) def test_map_with_side_inputs(self): - def mult_by(x, y): return x * y @@ -67,7 +63,6 @@ def mult_by(x, y): assert_that(pcoll, equal_to([3])) def test_map_with_named_side_inputs(self): - def mult_by(x, y): return x * y diff --git a/sdks/python/apache_beam/runners/dask/overrides.py b/sdks/python/apache_beam/runners/dask/overrides.py index 7bcfb1d19a79..d07c7cd518af 100644 --- a/sdks/python/apache_beam/runners/dask/overrides.py +++ b/sdks/python/apache_beam/runners/dask/overrides.py @@ -44,7 +44,6 @@ def get_windowing(self, inputs: t.Any) -> beam.Windowing: @typehints.with_input_types(K) @typehints.with_output_types(K) class _Reshuffle(beam.PTransform): - def expand(self, input_or_inputs): return beam.pvalue.PCollection.from_(input_or_inputs) @@ -60,7 +59,6 @@ def expand(self, input_or_inputs): @typehints.with_input_types(t.Tuple[K, V]) @typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) class _GroupByKeyOnly(beam.PTransform): - def expand(self, input_or_inputs): return beam.pvalue.PCollection.from_(input_or_inputs) @@ -76,7 +74,6 @@ def infer_output_type(self, input_type): @typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) class _GroupAlsoByWindow(beam.ParDo): """Not used yet...""" - def __init__(self, windowing): super().__init__(_GroupAlsoByWindowDoFn(windowing)) self.windowing = windowing @@ -88,22 +85,18 @@ def expand(self, input_or_inputs): @typehints.with_input_types(t.Tuple[K, V]) @typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) class _GroupByKey(beam.PTransform): - def expand(self, input_or_inputs): return input_or_inputs | "GroupByKey" >> _GroupByKeyOnly() class _Flatten(beam.PTransform): - def expand(self, input_or_inputs): is_bounded = all(pcoll.is_bounded for pcoll in input_or_inputs) return beam.pvalue.PCollection(self.pipeline, is_bounded=is_bounded) def dask_overrides() -> t.List[PTransformOverride]: - class CreateOverride(PTransformOverride): - def matches(self, applied_ptransform: AppliedPTransform) -> bool: return applied_ptransform.transform.__class__ == beam.Create @@ -112,7 +105,6 @@ def get_replacement_transform_for_applied_ptransform( return _Create(t.cast(beam.Create, applied_ptransform.transform).values) class ReshuffleOverride(PTransformOverride): - def matches(self, applied_ptransform: AppliedPTransform) -> bool: return applied_ptransform.transform.__class__ == beam.Reshuffle @@ -121,7 +113,6 @@ def get_replacement_transform_for_applied_ptransform( return _Reshuffle() class ReadOverride(PTransformOverride): - def matches(self, applied_ptransform: AppliedPTransform) -> bool: return applied_ptransform.transform.__class__ == beam.io.Read @@ -130,7 +121,6 @@ def get_replacement_transform_for_applied_ptransform( return _Read(t.cast(beam.io.Read, applied_ptransform.transform).source) class GroupByKeyOverride(PTransformOverride): - def matches(self, applied_ptransform: AppliedPTransform) -> bool: return applied_ptransform.transform.__class__ == beam.GroupByKey @@ -139,7 +129,6 @@ def get_replacement_transform_for_applied_ptransform( return _GroupByKey() class FlattenOverride(PTransformOverride): - def matches(self, applied_ptransform: AppliedPTransform) -> bool: return applied_ptransform.transform.__class__ == beam.Flatten diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 65635952bdbc..d2c5aff6a7ab 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -50,13 +50,11 @@ def apply(self, input_bag: OpInput) -> db.Bag: class NoOp(DaskBagOp): - def apply(self, input_bag: OpInput) -> db.Bag: return input_bag class Create(DaskBagOp): - def apply(self, input_bag: OpInput) -> db.Bag: assert input_bag is None, 'Create expects no input!' original_transform = t.cast(_Create, self.applied.transform) @@ -65,7 +63,6 @@ def apply(self, input_bag: OpInput) -> db.Bag: class ParDo(DaskBagOp): - def apply(self, input_bag: db.Bag) -> db.Bag: transform = t.cast(apache_beam.ParDo, self.applied.transform) return input_bag.map( @@ -73,7 +70,6 @@ def apply(self, input_bag: db.Bag) -> db.Bag: class Map(DaskBagOp): - def apply(self, input_bag: db.Bag) -> db.Bag: transform = t.cast(apache_beam.Map, self.applied.transform) return input_bag.map( @@ -81,9 +77,7 @@ def apply(self, input_bag: db.Bag) -> db.Bag: class GroupByKey(DaskBagOp): - def apply(self, input_bag: db.Bag) -> db.Bag: - def key(item): return item[0] @@ -95,7 +89,6 @@ def value(item): class Flatten(DaskBagOp): - def apply(self, input_bag: OpInput) -> db.Bag: assert type(input_bag) is list, 'Must take a sequence of bags!' return db.concat(input_bag) From 9fb52e5de1ef17e4f16975f27f9f5cdc6d318d4e Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Thu, 13 Oct 2022 14:22:45 -0700 Subject: [PATCH 090/173] Change an import to pass CI. --- .../apache_beam/runners/dask/dask_runner_test.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index f6a59e0dc3fc..a75b1f2fb94a 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -16,17 +16,17 @@ # import unittest +try: + import dask # pylint: disable=unused-import +except (ImportError, ModuleNotFoundError): + raise unittest.SkipTest('Dask must be installed to run tests.') + import apache_beam as beam from apache_beam.runners.dask.dask_runner import DaskRunner from apache_beam.testing import test_pipeline from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to -try: - import dask # pylint: disable=unused-import -except (ImportError, ModuleNotFoundError): - raise unittest.SkipTest('Dask must be installed to run tests.') - class DaskRunnerRunPipelineTest(unittest.TestCase): """Test class used to introspect the dask runner via a debugger.""" From 91115e06ca6151745804e0c1a07e8b9abfaf9243 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Thu, 13 Oct 2022 14:37:17 -0700 Subject: [PATCH 091/173] WIP -- better structure in ParDo for windowing. Thanks @pabloem. --- .../runners/dask/transform_evaluator.py | 24 ++++++++----------- 1 file changed, 10 insertions(+), 14 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 71cf4eb21e34..688c2743d6b9 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -194,23 +194,19 @@ def apply(self, input_bag: OpInput) -> db.Bag: user_state_context=None, bundle_finalizer_param=bundle_finalizer_param) - # Invoke setup just in case - do_fn_invoker.invoke_setup() - do_fn_invoker.invoke_start_bundle() - - for input_item in batch: - windowed_value = get_windowed_value(input_item, window_fn) - do_fn_invoker.invoke_process(windowed_value) + try: + # Invoke setup just in case + do_fn_invoker.invoke_setup() + do_fn_invoker.invoke_start_bundle() + return input_bag.map(get_windowed_value, window_fn).map(do_fn_invoker.invoke_process).flatten() - do_fn_invoker.invoke_finish_bundle() - # Invoke teardown just in case - do_fn_invoker.invoke_teardown() + # TODO(alxr): Check that finally will still be executed in the return. + finally: + do_fn_invoker.invoke_finish_bundle() + # Invoke teardown just in case + do_fn_invoker.invoke_teardown() - # This has to happen last as we might receive results - # in invoke_finish_bundle() or invoke_teardown() - ret = list(values) - return input_bag.map(transform.fn.process, *args, **kwargs).flatten() class Map(DaskBagOp): From 26c6016be707ad7b114f41e8ea24dad643e5286a Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Thu, 13 Oct 2022 14:38:03 -0700 Subject: [PATCH 092/173] Skip isort error; needed to get CI to pass. --- sdks/python/apache_beam/runners/dask/dask_runner_test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index a75b1f2fb94a..4be1341f5df3 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -17,7 +17,7 @@ import unittest try: - import dask # pylint: disable=unused-import + import dask # pylint: disable=unused-import # isort:skip except (ImportError, ModuleNotFoundError): raise unittest.SkipTest('Dask must be installed to run tests.') From aec19bfdeb709a50b3a1e0d7392cf7fedb91a903 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Thu, 13 Oct 2022 16:29:08 -0700 Subject: [PATCH 093/173] Skip test logic may favor better with isort. --- sdks/python/apache_beam/runners/dask/dask_runner_test.py | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index 4be1341f5df3..3ad2a8bde586 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -16,13 +16,11 @@ # import unittest +import apache_beam as beam try: - import dask # pylint: disable=unused-import # isort:skip + from apache_beam.runners.dask.dask_runner import DaskRunner except (ImportError, ModuleNotFoundError): raise unittest.SkipTest('Dask must be installed to run tests.') - -import apache_beam as beam -from apache_beam.runners.dask.dask_runner import DaskRunner from apache_beam.testing import test_pipeline from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to From 067323562ef3c4b56ebdf16336c2e989de3ee80b Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Thu, 13 Oct 2022 16:39:08 -0700 Subject: [PATCH 094/173] (Maybe) the last isort fix. --- sdks/python/apache_beam/runners/dask/dask_runner_test.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index 3ad2a8bde586..3c613efb502e 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -17,13 +17,14 @@ import unittest import apache_beam as beam +from apache_beam.testing import test_pipeline +from apache_beam.testing.util import assert_that +from apache_beam.testing.util import equal_to + try: from apache_beam.runners.dask.dask_runner import DaskRunner except (ImportError, ModuleNotFoundError): raise unittest.SkipTest('Dask must be installed to run tests.') -from apache_beam.testing import test_pipeline -from apache_beam.testing.util import assert_that -from apache_beam.testing.util import equal_to class DaskRunnerRunPipelineTest(unittest.TestCase): From de03a32572fa19d0529dbeaada683ae16cb5109b Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Thu, 13 Oct 2022 17:54:59 -0700 Subject: [PATCH 095/173] Tested pipeline options (added one fix). --- .../apache_beam/runners/dask/dask_runner.py | 1 + .../runners/dask/dask_runner_test.py | 36 +++++++++++++++++++ 2 files changed, 37 insertions(+) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index e0bd71f8c9f6..109c4379b45d 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -63,6 +63,7 @@ def _add_argparse_args(cls, parser: argparse.ArgumentParser) -> None: help='Timeout duration for initial connection to the scheduler.') parser.add_argument( '--dask_scheduler_file', + dest='scheduler_file', type=str, default=None, help='Path to a file with scheduler information if available.') diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index 3c613efb502e..2526d39870f6 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -17,16 +17,52 @@ import unittest import apache_beam as beam +from apache_beam.options.pipeline_options import PipelineOptions from apache_beam.testing import test_pipeline from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to try: + from apache_beam.runners.dask.dask_runner import DaskOptions from apache_beam.runners.dask.dask_runner import DaskRunner + import dask + import dask.distributed as ddist except (ImportError, ModuleNotFoundError): raise unittest.SkipTest('Dask must be installed to run tests.') +class DaskOptionsTest(unittest.TestCase): + + def test_parses_connection_timeout__defaults_to_none(self): + default_options = PipelineOptions([]) + default_dask_options = default_options.view_as(DaskOptions) + self.assertEqual(None, default_dask_options.timeout) + + def test_parses_connection_timeout__parses_int(self): + conn_options = PipelineOptions('--dask_connection_timeout 12'.split()) + dask_conn_options = conn_options.view_as(DaskOptions) + self.assertEqual(12, dask_conn_options.timeout) + + def test_parses_connection_timeout__handles_bad_input(self): + err_options = PipelineOptions('--dask_connection_timeout foo'.split()) + dask_err_options = err_options.view_as(DaskOptions) + self.assertEqual(dask.config.no_default, dask_err_options.timeout) + + def test_parser_destinations__agree_with_dask_client(self): + options = PipelineOptions( + '--dask_client_address localhost:8080 --dask_connection_timeout 7 ' + '--dask_scheduler_file foobar.cfg --dask_client_name charlie ' + '--dask_connection_limit 1024'.split() + ) + dask_options = options.view_as(DaskOptions) + + try: + client = ddist.Client(**dask_options.get_all_options(drop_default=True)) + client.close() + except ValueError: + self.fail('parsed args did not match dask Client args.') + + class DaskRunnerRunPipelineTest(unittest.TestCase): """Test class used to introspect the dask runner via a debugger.""" def setUp(self) -> None: From 7e0a2c77f2d3a4d480b93379e172c9ba6d4e5a09 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Thu, 13 Oct 2022 18:00:32 -0700 Subject: [PATCH 096/173] Improve formatting of test. --- sdks/python/apache_beam/runners/dask/dask_runner_test.py | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index 2526d39870f6..e25b4109566b 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -32,7 +32,6 @@ class DaskOptionsTest(unittest.TestCase): - def test_parses_connection_timeout__defaults_to_none(self): default_options = PipelineOptions([]) default_dask_options = default_options.view_as(DaskOptions) @@ -50,10 +49,9 @@ def test_parses_connection_timeout__handles_bad_input(self): def test_parser_destinations__agree_with_dask_client(self): options = PipelineOptions( - '--dask_client_address localhost:8080 --dask_connection_timeout 7 ' - '--dask_scheduler_file foobar.cfg --dask_client_name charlie ' - '--dask_connection_limit 1024'.split() - ) + '--dask_client_address localhost:8080 --dask_connection_timeout 7 ' + '--dask_scheduler_file foobar.cfg --dask_client_name charlie ' + '--dask_connection_limit 1024'.split()) dask_options = options.view_as(DaskOptions) try: From 39b1e1c68346f969f109faaacf3e88cd11c63f4e Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Thu, 13 Oct 2022 18:43:38 -0700 Subject: [PATCH 097/173] Self-review: removing side inputs. In addition, adding a more helpful property to the base DaskBagOp (tranform). --- .../apache_beam/runners/dask/dask_runner_test.py | 16 ---------------- .../runners/dask/transform_evaluator.py | 10 +++++----- 2 files changed, 5 insertions(+), 21 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index e25b4109566b..375d3c023b30 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -87,22 +87,6 @@ def double(x): pcoll = p | beam.Create([1]) | beam.Map(double) | beam.GroupByKey() assert_that(pcoll, equal_to([(2, [1])])) - def test_map_with_side_inputs(self): - def mult_by(x, y): - return x * y - - with self.pipeline as p: - pcoll = p | beam.Create([1]) | beam.Map(mult_by, 3) - assert_that(pcoll, equal_to([3])) - - def test_map_with_named_side_inputs(self): - def mult_by(x, y): - return x * y - - with self.pipeline as p: - pcoll = p | beam.Create([1]) | beam.Map(mult_by, y=3) - assert_that(pcoll, equal_to([3])) - if __name__ == '__main__': unittest.main() diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index d2c5aff6a7ab..c4aac7f2111f 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -41,8 +41,8 @@ class DaskBagOp(abc.ABC): applied: AppliedPTransform @property - def side_inputs(self): - return self.applied.transform.args + def transform(self): + return self.applied.transform @abc.abstractmethod def apply(self, input_bag: OpInput) -> db.Bag: @@ -57,21 +57,21 @@ def apply(self, input_bag: OpInput) -> db.Bag: class Create(DaskBagOp): def apply(self, input_bag: OpInput) -> db.Bag: assert input_bag is None, 'Create expects no input!' - original_transform = t.cast(_Create, self.applied.transform) + original_transform = t.cast(_Create, self.transform) items = original_transform.values return db.from_sequence(items) class ParDo(DaskBagOp): def apply(self, input_bag: db.Bag) -> db.Bag: - transform = t.cast(apache_beam.ParDo, self.applied.transform) + transform = t.cast(apache_beam.ParDo, self.transform) return input_bag.map( transform.fn.process, *transform.args, **transform.kwargs).flatten() class Map(DaskBagOp): def apply(self, input_bag: db.Bag) -> db.Bag: - transform = t.cast(apache_beam.Map, self.applied.transform) + transform = t.cast(apache_beam.Map, self.transform) return input_bag.map( transform.fn.process, *transform.args, **transform.kwargs) From 6db49fad70e1493b39970cdc99f98c2bdf196300 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Mon, 17 Oct 2022 14:08:26 -0700 Subject: [PATCH 098/173] add dask to coverage suite in tox. --- sdks/python/tox.ini | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini index 7e92ce28f85f..828c131dce01 100644 --- a/sdks/python/tox.ini +++ b/sdks/python/tox.ini @@ -101,7 +101,7 @@ deps = codecov pytest-cov==3.0.0 passenv = GIT_* BUILD_* ghprb* CHANGE_ID BRANCH_NAME JENKINS_* CODECOV_* -extras = test,gcp,interactive,dataframe,aws +extras = test,gcp,interactive,dataframe,aws,dask commands = -rm .coverage {toxinidir}/scripts/run_pytest.sh {envname} "{posargs}" "--cov-report=xml --cov=. --cov-append" From ed00139832094d2eee9e298666935d75adf6a7cf Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Mon, 17 Oct 2022 14:33:35 -0700 Subject: [PATCH 099/173] Experiment: Windowed ParDo with @pabloem. --- .../runners/dask/transform_evaluator.py | 103 ++++-------------- 1 file changed, 20 insertions(+), 83 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 688c2743d6b9..b87c49785df7 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -21,52 +21,35 @@ TODO(alxr): Translate ops from https://docs.dask.org/en/latest/bag-api.html. """ -import contextlib +import abc import dataclasses +import typing as t +from dataclasses import field -import abc import dask.bag as db -import typing as t -import functools import apache_beam from apache_beam import TaggedOutput, DoFn from apache_beam.internal import util from apache_beam.pipeline import AppliedPTransform -from apache_beam.runners.common import DoFnContext, DoFnSignature, Receiver, _OutputHandler, DoFnInvoker +from apache_beam.runners.common import DoFnContext +from apache_beam.runners.common import DoFnSignature +from apache_beam.runners.common import Receiver +from apache_beam.runners.common import _OutputHandler +from apache_beam.runners.common import DoFnInvoker from apache_beam.runners.dask.overrides import _Create from apache_beam.runners.dask.overrides import _Flatten from apache_beam.runners.dask.overrides import _GroupByKeyOnly -from apache_beam.transforms.sideinputs import SideInputMap -from apache_beam.transforms.window import WindowFn, TimestampedValue, GlobalWindow from apache_beam.utils.windowed_value import WindowedValue OpInput = t.Union[db.Bag, t.Sequence[db.Bag], None] - -@dataclasses.dataclass -class WindowAccessor: - window_fn: WindowFn - - def __getitem__(self, item: t.Any): - if isinstance(item, TaggedOutput): - item = item.value - - if isinstance(item, WindowedValue): - windowed_value = item - elif isinstance(item, TimestampedValue): - assign_context = WindowFn.AssignContext(item.timestamp, item.value) - windowed_value = WindowedValue(item.value, item.timestamp, - self.window_fn.assign(assign_context)) - else: - windowed_value = WindowedValue(item, 0, (GlobalWindow(),)) - - return windowed_value +PCollVal = t.Union[WindowedValue, t.Any] @dataclasses.dataclass class TaggingReceiver(Receiver): tag: str - values: t.List[t.Union[WindowedValue, t.Any]] + values: t.List[PCollVal] def receive(self, windowed_value: WindowedValue): if self.tag: @@ -78,7 +61,9 @@ def receive(self, windowed_value: WindowedValue): @dataclasses.dataclass class OneReceiver(dict): - values: t.List[t.Union[WindowedValue, t.Any]] + values: t.List[PCollVal] = field( + default_factory=list + ) def __missing__(self, key): if key not in self: @@ -86,52 +71,6 @@ def __missing__(self, key): return self[key] -@dataclasses.dataclass -class DoFnWorker: - label: str - map_fn: DoFn - window_fn: WindowFn - side_inputs: t.List[SideInputMap] - args: t.Any - kwargs: t.Any - - def __post_init__(self): - self._values = [] - - tagged_receivers = OneReceiver(self._values) - do_fn_signature = DoFnSignature(self.map_fn) - output_handler = _OutputHandler( - window_fn=self.window_fn, - main_receivers=tagged_receivers[None], - tagged_receivers=tagged_receivers, - per_element_output_counter=None, - ) - - self._invoker = DoFnInvoker.create_invoker( - do_fn_signature, - output_handler, - DoFnContext(self.label, state=None), - self.side_inputs, - self.args, - self.kwargs, - user_state_context=None, - bundle_finalizer_param=DoFn.BundleFinalizerParam(), - ) - - def __del__(self): - self._invoker.invoke_teardown() - - def invoke(self, items): - try: - self._invoker.invoke_setup() - self._invoker.invoke_start_bundle() - - self._invoker.invoke_process() - - finally: - self._invoker.invoke_finish_bundle() - - @dataclasses.dataclass class DaskBagOp(abc.ABC): applied: AppliedPTransform @@ -173,9 +112,7 @@ def apply(self, input_bag: OpInput) -> db.Bag: bundle_finalizer_param = DoFn.BundleFinalizerParam() do_fn_signature = DoFnSignature(map_fn) - values = [] - - tagged_receivers = OneReceiver(values) + tagged_receivers = OneReceiver() output_processor = _OutputHandler( window_fn=window_fn, @@ -194,19 +131,19 @@ def apply(self, input_bag: OpInput) -> db.Bag: user_state_context=None, bundle_finalizer_param=bundle_finalizer_param) - try: - # Invoke setup just in case + def apply_dofn_to_bundle(items): do_fn_invoker.invoke_setup() do_fn_invoker.invoke_start_bundle() - return input_bag.map(get_windowed_value, window_fn).map(do_fn_invoker.invoke_process).flatten() - # TODO(alxr): Check that finally will still be executed in the return. - finally: + results = [do_fn_invoker.invoke_process(it) for it in items] + results.extend(tagged_receivers.values) + do_fn_invoker.invoke_finish_bundle() - # Invoke teardown just in case do_fn_invoker.invoke_teardown() + return results + return input_bag.map_partitions(apply_dofn_to_bundle).flatten() class Map(DaskBagOp): From d35e9d612ca3c088a1efc7a1149837dd05bb7941 Mon Sep 17 00:00:00 2001 From: Pablo E Date: Mon, 17 Oct 2022 14:42:56 -0700 Subject: [PATCH 100/173] add mandatory args for _OutputHandler --- sdks/python/apache_beam/runners/dask/transform_evaluator.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index b87c49785df7..80565c91276f 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -119,6 +119,9 @@ def apply(self, input_bag: OpInput) -> db.Bag: main_receivers=tagged_receivers[None], tagged_receivers=tagged_receivers, per_element_output_counter=None, + output_batch_converter=None, + process_yields_batches=False, + process_batch_yields_elements=False ) do_fn_invoker = DoFnInvoker.create_invoker( From 3feeeac6188a1f2f3f7f64e474029e23dde2884e Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Mon, 17 Oct 2022 14:51:25 -0700 Subject: [PATCH 101/173] Update: still need to pre-apply windowed values. --- .../runners/dask/transform_evaluator.py | 24 +++++++++++++++++-- 1 file changed, 22 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 80565c91276f..7f750dfc9a60 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -29,7 +29,8 @@ import dask.bag as db import apache_beam -from apache_beam import TaggedOutput, DoFn +from apache_beam import TaggedOutput +from apache_beam import DoFn from apache_beam.internal import util from apache_beam.pipeline import AppliedPTransform from apache_beam.runners.common import DoFnContext @@ -40,12 +41,31 @@ from apache_beam.runners.dask.overrides import _Create from apache_beam.runners.dask.overrides import _Flatten from apache_beam.runners.dask.overrides import _GroupByKeyOnly +from apache_beam.transforms.window import TimestampedValue +from apache_beam.transforms.window import WindowFn +from apache_beam.transforms.window import GlobalWindow from apache_beam.utils.windowed_value import WindowedValue OpInput = t.Union[db.Bag, t.Sequence[db.Bag], None] PCollVal = t.Union[WindowedValue, t.Any] +def get_windowed_value(item: t.Any, window_fn: WindowFn) -> WindowedValue: + if isinstance(item, TaggedOutput): + item = item.value + + if isinstance(item, WindowedValue): + windowed_value = item + elif isinstance(item, TimestampedValue): + assign_context = WindowFn.AssignContext(item.timestamp, item.value) + windowed_value = WindowedValue(item.value, item.timestamp, + tuple(window_fn.assign(assign_context))) + else: + windowed_value = WindowedValue(item, 0, (GlobalWindow(),)) + + return windowed_value + + @dataclasses.dataclass class TaggingReceiver(Receiver): tag: str @@ -146,7 +166,7 @@ def apply_dofn_to_bundle(items): return results - return input_bag.map_partitions(apply_dofn_to_bundle).flatten() + return input_bag.map(get_windowed_value, window_fn).map_partitions(apply_dofn_to_bundle).flatten() class Map(DaskBagOp): From 191580d3786bc2c651e68f7956c8fbb9e808f85e Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Tue, 18 Oct 2022 10:45:59 -0700 Subject: [PATCH 102/173] Capture value error in assert. --- sdks/python/apache_beam/runners/dask/dask_runner_test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index 375d3c023b30..a3f38139b856 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -57,8 +57,8 @@ def test_parser_destinations__agree_with_dask_client(self): try: client = ddist.Client(**dask_options.get_all_options(drop_default=True)) client.close() - except ValueError: - self.fail('parsed args did not match dask Client args.') + except ValueError as e: + self.fail(f'parsed args did not match dask Client args: {e!s}') class DaskRunnerRunPipelineTest(unittest.TestCase): From 085447e1bff046fa9610aff1a72ea932465183fe Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Tue, 18 Oct 2022 15:22:09 -0700 Subject: [PATCH 103/173] Change timeout value to 600 seconds. --- sdks/python/apache_beam/runners/dask/dask_runner_test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index a3f38139b856..9a610698f2ba 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -49,7 +49,7 @@ def test_parses_connection_timeout__handles_bad_input(self): def test_parser_destinations__agree_with_dask_client(self): options = PipelineOptions( - '--dask_client_address localhost:8080 --dask_connection_timeout 7 ' + '--dask_client_address localhost:8080 --dask_connection_timeout 600 ' '--dask_scheduler_file foobar.cfg --dask_client_name charlie ' '--dask_connection_limit 1024'.split()) dask_options = options.view_as(DaskOptions) From 1a60a5e641283cc9d02c72a1c62cb76e93095837 Mon Sep 17 00:00:00 2001 From: Pablo Date: Fri, 21 Oct 2022 00:57:46 -0400 Subject: [PATCH 104/173] ignoring broken test --- sdks/python/apache_beam/runners/dask/dask_runner_test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index 9a610698f2ba..ade456483d0c 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -47,6 +47,7 @@ def test_parses_connection_timeout__handles_bad_input(self): dask_err_options = err_options.view_as(DaskOptions) self.assertEqual(dask.config.no_default, dask_err_options.timeout) + @unittest.skip("Buggy test. TODO(alxmrs): Fix") def test_parser_destinations__agree_with_dask_client(self): options = PipelineOptions( '--dask_client_address localhost:8080 --dask_connection_timeout 600 ' From c1037f841cd6d9cd4527ec73054d17eafe6c8a3f Mon Sep 17 00:00:00 2001 From: Pablo Date: Fri, 21 Oct 2022 00:59:08 -0400 Subject: [PATCH 105/173] Update CHANGES.md --- CHANGES.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/CHANGES.md b/CHANGES.md index 549a27e6ade9..e01312c44a5d 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -54,6 +54,7 @@ ## Highlights * Python 3.10 support in Apache Beam ([#21458](https://github.com/apache/beam/issues/21458)). +* An initial implementation of a runner that allows us to run Beam pipelines on Dask. Try it out and give us feedback! (Python) ([#18962](https://github.com/apache/beam/issues/18962)). ## I/Os @@ -71,7 +72,7 @@ * X feature added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). * Dataframe wrapper added in Go SDK via Cross-Language (Need to manually start python expansion service). (Go) ([#23384](https://github.com/apache/beam/issues/23384)). * Name all Java threads to aid in debugging ([#23049](https://github.com/apache/beam/issues/23049)). -* Dask support for the Python SDK (Python) ([#18962](https://github.com/apache/beam/issues/18962)). +* An initial implementation of a runner that allows us to run Beam pipelines on Dask. (Python) ([#18962](https://github.com/apache/beam/issues/18962)). ## Breaking Changes From 9e79ffd805535e8511406d81cf7e7042f93abe83 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Mon, 24 Oct 2022 12:57:30 -0700 Subject: [PATCH 106/173] Using reflection to test the Dask client constructor. --- .../apache_beam/runners/dask/dask_runner_test.py | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index ade456483d0c..ed8bf2b0f9bf 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -47,7 +47,6 @@ def test_parses_connection_timeout__handles_bad_input(self): dask_err_options = err_options.view_as(DaskOptions) self.assertEqual(dask.config.no_default, dask_err_options.timeout) - @unittest.skip("Buggy test. TODO(alxmrs): Fix") def test_parser_destinations__agree_with_dask_client(self): options = PipelineOptions( '--dask_client_address localhost:8080 --dask_connection_timeout 600 ' @@ -55,11 +54,12 @@ def test_parser_destinations__agree_with_dask_client(self): '--dask_connection_limit 1024'.split()) dask_options = options.view_as(DaskOptions) - try: - client = ddist.Client(**dask_options.get_all_options(drop_default=True)) - client.close() - except ValueError as e: - self.fail(f'parsed args did not match dask Client args: {e!s}') + # Get the argument names for the constructor. + client_args = ddist.Client.__init__.__code__.co_varnames + + for opt_name in dask_options.get_all_options(drop_default=True).keys(): + with self.subTest(f'{opt_name} in dask.distributed.Client constructor'): + self.assertIn(opt_name, client_args) class DaskRunnerRunPipelineTest(unittest.TestCase): From 4edc9709b31124306a65d1ee11529db6ac3f7771 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Mon, 24 Oct 2022 13:22:18 -0700 Subject: [PATCH 107/173] Better method of inspecting the constructor parameters (thanks @TomAugspurger!). --- sdks/python/apache_beam/runners/dask/dask_runner_test.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index 498be37b7e04..8b36e2ed3928 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -14,6 +14,7 @@ # See the License for the specific language governing permissions and # limitations under the License. # +import inspect import unittest import apache_beam as beam @@ -55,7 +56,7 @@ def test_parser_destinations__agree_with_dask_client(self): dask_options = options.view_as(DaskOptions) # Get the argument names for the constructor. - client_args = ddist.Client.__init__.__code__.co_varnames + client_args = list(inspect.signature(ddist.Client).parameters) for opt_name in dask_options.get_all_options(drop_default=True).keys(): with self.subTest(f'{opt_name} in dask.distributed.Client constructor'): From 36bea9afdfddcfd3850e2735ea9cbae0d4bb7a32 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Mon, 31 Oct 2022 15:58:14 -0700 Subject: [PATCH 108/173] Minor fixes, and now unit tests are passing!! --- sdks/python/apache_beam/runners/dask/transform_evaluator.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index b083fd74d10e..cf69dacb5e63 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -126,6 +126,7 @@ def apply(self, input_bag: OpInput) -> db.Bag: label = transform.label map_fn = transform.fn args, kwargs = transform.raw_side_inputs + args = list(args) main_input = next(iter(self.applied.main_inputs.values())) window_fn = main_input.windowing.windowfn if hasattr(main_input, "windowing") else None @@ -159,8 +160,8 @@ def apply_dofn_to_bundle(items): do_fn_invoker.invoke_setup() do_fn_invoker.invoke_start_bundle() - results = [do_fn_invoker.invoke_process(it) for it in items] - results.extend(tagged_receivers.values) + to_proc = list(items) + tagged_receivers.values + results = [do_fn_invoker.invoke_process(it) for it in to_proc] do_fn_invoker.invoke_finish_bundle() do_fn_invoker.invoke_teardown() From df315c1769ebcbf89a7d1a774ab12008d2e3eaa4 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Tue, 1 Nov 2022 09:24:30 -0700 Subject: [PATCH 109/173] Ran yapf on Dask sources. --- .../apache_beam/runners/dask/dask_runner.py | 1 + .../runners/dask/dask_runner_test.py | 10 +++- .../apache_beam/runners/dask/overrides.py | 23 ++++--- .../runners/dask/transform_evaluator.py | 60 ++++++++++--------- 4 files changed, 57 insertions(+), 37 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index 489d24a88b3e..fd1fee55854d 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -125,6 +125,7 @@ def metrics(self): class DaskRunner(BundleBasedDirectRunner): """Executes a pipeline on a Dask distributed client.""" + @staticmethod def to_dask_bag_visitor() -> PipelineVisitor: from dask import bag as db diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index 8b36e2ed3928..97083c11e3c0 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -33,6 +33,7 @@ class DaskOptionsTest(unittest.TestCase): + def test_parses_connection_timeout__defaults_to_none(self): default_options = PipelineOptions([]) default_dask_options = default_options.view_as(DaskOptions) @@ -65,6 +66,7 @@ def test_parser_destinations__agree_with_dask_client(self): class DaskRunnerRunPipelineTest(unittest.TestCase): """Test class used to introspect the dask runner via a debugger.""" + def setUp(self) -> None: self.pipeline = test_pipeline.TestPipeline(runner=DaskRunner()) @@ -74,6 +76,7 @@ def test_create(self): assert_that(pcoll, equal_to([1])) def test_create_and_map(self): + def double(x): return x * 2 @@ -82,16 +85,16 @@ def double(x): assert_that(pcoll, equal_to([2])) def test_create_map_and_groupby(self): + def double(x): return x * 2, x with self.pipeline as p: pcoll = p | beam.Create([1]) | beam.Map(double) | beam.GroupByKey() - assert_that(pcoll, equal_to([ - (2, [1]) - ])) + assert_that(pcoll, equal_to([(2, [1])])) def test_map_with_side_inputs(self): + def mult_by(x, y): return x * y @@ -100,6 +103,7 @@ def mult_by(x, y): assert_that(pcoll, equal_to([3])) def test_map_with_named_side_inputs(self): + def mult_by(x, y): return x * y diff --git a/sdks/python/apache_beam/runners/dask/overrides.py b/sdks/python/apache_beam/runners/dask/overrides.py index ab159c7c7ece..c5bf4d862bad 100644 --- a/sdks/python/apache_beam/runners/dask/overrides.py +++ b/sdks/python/apache_beam/runners/dask/overrides.py @@ -44,6 +44,7 @@ def get_windowing(self, inputs: t.Any) -> beam.Windowing: @typehints.with_input_types(K) @typehints.with_output_types(K) class _Reshuffle(beam.PTransform): + def expand(self, input_or_inputs): return beam.pvalue.PCollection.from_(input_or_inputs) @@ -59,6 +60,7 @@ def expand(self, input_or_inputs): @typehints.with_input_types(t.Tuple[K, V]) @typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) class _GroupByKeyOnly(beam.PTransform): + def expand(self, input_or_inputs): return beam.pvalue.PCollection.from_(input_or_inputs) @@ -73,6 +75,7 @@ def infer_output_type(self, input_type): @typehints.with_input_types(t.Tuple[K, t.Iterable[V]]) @typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) class _GroupAlsoByWindow(beam.ParDo): + def __init__(self, windowing): super().__init__(_GroupAlsoByWindowDoFn(windowing)) self.windowing = windowing @@ -90,18 +93,20 @@ def expand(self, input_or_inputs): input_or_inputs | "ReifyWindows" >> beam.ParDo(beam.GroupByKey.ReifyWindows()) | "GroupByKey" >> _GroupByKeyOnly() - | "GroupByWindow" >> _GroupAlsoByWindow(input_or_inputs.windowing) - ) + | "GroupByWindow" >> _GroupAlsoByWindow(input_or_inputs.windowing)) class _Flatten(beam.PTransform): + def expand(self, input_or_inputs): is_bounded = all(pcoll.is_bounded for pcoll in input_or_inputs) return beam.pvalue.PCollection(self.pipeline, is_bounded=is_bounded) def dask_overrides() -> t.List[PTransformOverride]: + class CreateOverride(PTransformOverride): + def matches(self, applied_ptransform: AppliedPTransform) -> bool: return applied_ptransform.transform.__class__ == beam.Create @@ -110,6 +115,7 @@ def get_replacement_transform_for_applied_ptransform( return _Create(t.cast(beam.Create, applied_ptransform.transform).values) class ReshuffleOverride(PTransformOverride): + def matches(self, applied_ptransform: AppliedPTransform) -> bool: return applied_ptransform.transform.__class__ == beam.Reshuffle @@ -118,6 +124,7 @@ def get_replacement_transform_for_applied_ptransform( return _Reshuffle() class ReadOverride(PTransformOverride): + def matches(self, applied_ptransform: AppliedPTransform) -> bool: return applied_ptransform.transform.__class__ == beam.io.Read @@ -126,6 +133,7 @@ def get_replacement_transform_for_applied_ptransform( return _Read(t.cast(beam.io.Read, applied_ptransform.transform).source) class GroupByKeyOverride(PTransformOverride): + def matches(self, applied_ptransform: AppliedPTransform) -> bool: return applied_ptransform.transform.__class__ == beam.GroupByKey @@ -134,6 +142,7 @@ def get_replacement_transform_for_applied_ptransform( return _GroupByKey() class FlattenOverride(PTransformOverride): + def matches(self, applied_ptransform: AppliedPTransform) -> bool: return applied_ptransform.transform.__class__ == beam.Flatten @@ -142,9 +151,9 @@ def get_replacement_transform_for_applied_ptransform( return _Flatten() return [ - CreateOverride(), - ReshuffleOverride(), - ReadOverride(), - GroupByKeyOverride(), - FlattenOverride(), + CreateOverride(), + ReshuffleOverride(), + ReadOverride(), + GroupByKeyOverride(), + FlattenOverride(), ] diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index cf69dacb5e63..75a3dfb276cf 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -59,10 +59,10 @@ def get_windowed_value(item: t.Any, window_fn: WindowFn) -> WindowedValue: windowed_value = item elif isinstance(item, TimestampedValue): assign_context = WindowFn.AssignContext(item.timestamp, item.value) - windowed_value = WindowedValue(item.value, item.timestamp, - tuple(window_fn.assign(assign_context))) + windowed_value = WindowedValue( + item.value, item.timestamp, tuple(window_fn.assign(assign_context))) else: - windowed_value = WindowedValue(item, 0, (GlobalWindow(),)) + windowed_value = WindowedValue(item, 0, (GlobalWindow(), )) return windowed_value @@ -82,9 +82,7 @@ def receive(self, windowed_value: WindowedValue): @dataclasses.dataclass class OneReceiver(dict): - values: t.List[PCollVal] = field( - default_factory=list - ) + values: t.List[PCollVal] = field(default_factory=list) def __missing__(self, key): if key not in self: @@ -106,11 +104,13 @@ def apply(self, input_bag: OpInput) -> db.Bag: class NoOp(DaskBagOp): + def apply(self, input_bag: OpInput) -> db.Bag: return input_bag class Create(DaskBagOp): + def apply(self, input_bag: OpInput) -> db.Bag: assert input_bag is None, 'Create expects no input!' original_transform = t.cast(_Create, self.transform) @@ -128,7 +128,8 @@ def apply(self, input_bag: OpInput) -> db.Bag: args, kwargs = transform.raw_side_inputs args = list(args) main_input = next(iter(self.applied.main_inputs.values())) - window_fn = main_input.windowing.windowfn if hasattr(main_input, "windowing") else None + window_fn = main_input.windowing.windowfn if hasattr( + main_input, "windowing") else None context = DoFnContext(label, state=None) bundle_finalizer_param = DoFn.BundleFinalizerParam() @@ -137,24 +138,23 @@ def apply(self, input_bag: OpInput) -> db.Bag: tagged_receivers = OneReceiver() output_processor = _OutputHandler( - window_fn=window_fn, - main_receivers=tagged_receivers[None], - tagged_receivers=tagged_receivers, - per_element_output_counter=None, - output_batch_converter=None, - process_yields_batches=False, - process_batch_yields_elements=False - ) + window_fn=window_fn, + main_receivers=tagged_receivers[None], + tagged_receivers=tagged_receivers, + per_element_output_counter=None, + output_batch_converter=None, + process_yields_batches=False, + process_batch_yields_elements=False) do_fn_invoker = DoFnInvoker.create_invoker( - do_fn_signature, - output_processor, - context, - None, - args, - kwargs, - user_state_context=None, - bundle_finalizer_param=bundle_finalizer_param) + do_fn_signature, + output_processor, + context, + None, + args, + kwargs, + user_state_context=None, + bundle_finalizer_param=bundle_finalizer_param) def apply_dofn_to_bundle(items): do_fn_invoker.invoke_setup() @@ -168,10 +168,13 @@ def apply_dofn_to_bundle(items): return results - return input_bag.map(get_windowed_value, window_fn).map_partitions(apply_dofn_to_bundle).flatten() + return input_bag.map( + get_windowed_value, + window_fn).map_partitions(apply_dofn_to_bundle).flatten() class Map(DaskBagOp): + def apply(self, input_bag: OpInput) -> db.Bag: transform = t.cast(apache_beam.Map, self.transform) args, kwargs = util.insert_values_in_args( @@ -180,7 +183,9 @@ def apply(self, input_bag: OpInput) -> db.Bag: class GroupByKey(DaskBagOp): + def apply(self, input_bag: OpInput) -> db.Bag: + def key(item): return item[0] @@ -192,6 +197,7 @@ def value(item): class Flatten(DaskBagOp): + def apply(self, input_bag: OpInput) -> db.Bag: assert type(input_bag) is list, 'Must take a sequence of bags!' return db.concat(input_bag) @@ -200,7 +206,7 @@ def apply(self, input_bag: OpInput) -> db.Bag: TRANSLATIONS = { _Create: Create, apache_beam.ParDo: ParDo, - apache_beam.Map: Map, - _GroupByKeyOnly: GroupByKey, - _Flatten: Flatten, + apache_beam.Map: Map, + _GroupByKeyOnly: GroupByKey, + _Flatten: Flatten, } From ef0d2b6a48bf42e3e0d33cb674945d1b05318df1 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Tue, 1 Nov 2022 09:30:00 -0700 Subject: [PATCH 110/173] Ran lint checks. --- sdks/python/apache_beam/runners/dask/dask_runner.py | 7 ++----- .../apache_beam/runners/dask/dask_runner_test.py | 5 +++-- sdks/python/apache_beam/runners/dask/overrides.py | 1 + .../apache_beam/runners/dask/transform_evaluator.py | 11 ++++++----- 4 files changed, 12 insertions(+), 12 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index fd1fee55854d..b0dff23a39ae 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -21,8 +21,9 @@ transformations across processes and workers via Dask distributed's scheduler. """ -import argparse import dataclasses + +import argparse import typing as t from apache_beam import pvalue @@ -35,11 +36,7 @@ from apache_beam.runners.direct.direct_runner import BundleBasedDirectRunner from apache_beam.runners.runner import PipelineResult from apache_beam.runners.runner import PipelineState -from apache_beam.transforms.window import WindowFn -from apache_beam.transforms.window import TimestampedValue -from apache_beam.transforms.window import GlobalWindow from apache_beam.utils.interactive_utils import is_in_notebook -from apache_beam.utils.windowed_value import WindowedValue class DaskOptions(PipelineOptions): diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index 97083c11e3c0..3ed34633107f 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -24,10 +24,11 @@ from apache_beam.testing.util import equal_to try: - from apache_beam.runners.dask.dask_runner import DaskOptions - from apache_beam.runners.dask.dask_runner import DaskRunner import dask import dask.distributed as ddist + + from apache_beam.runners.dask.dask_runner import DaskOptions + from apache_beam.runners.dask.dask_runner import DaskRunner except (ImportError, ModuleNotFoundError): raise unittest.SkipTest('Dask must be installed to run tests.') diff --git a/sdks/python/apache_beam/runners/dask/overrides.py b/sdks/python/apache_beam/runners/dask/overrides.py index c5bf4d862bad..2ff02549e41e 100644 --- a/sdks/python/apache_beam/runners/dask/overrides.py +++ b/sdks/python/apache_beam/runners/dask/overrides.py @@ -15,6 +15,7 @@ # limitations under the License. # import dataclasses + import typing as t import apache_beam as beam diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 75a3dfb276cf..eecedba24ed5 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -22,29 +22,30 @@ TODO(alxr): Translate ops from https://docs.dask.org/en/latest/bag-api.html. """ -import abc import dataclasses -import typing as t from dataclasses import field +import abc import dask.bag as db +import typing as t + import apache_beam -from apache_beam import TaggedOutput from apache_beam import DoFn +from apache_beam import TaggedOutput from apache_beam.internal import util from apache_beam.pipeline import AppliedPTransform from apache_beam.runners.common import DoFnContext +from apache_beam.runners.common import DoFnInvoker from apache_beam.runners.common import DoFnSignature from apache_beam.runners.common import Receiver from apache_beam.runners.common import _OutputHandler -from apache_beam.runners.common import DoFnInvoker from apache_beam.runners.dask.overrides import _Create from apache_beam.runners.dask.overrides import _Flatten from apache_beam.runners.dask.overrides import _GroupByKeyOnly +from apache_beam.transforms.window import GlobalWindow from apache_beam.transforms.window import TimestampedValue from apache_beam.transforms.window import WindowFn -from apache_beam.transforms.window import GlobalWindow from apache_beam.utils.windowed_value import WindowedValue OpInput = t.Union[db.Bag, t.Sequence[db.Bag], None] From 6c2cc4e53ec0d77ee0df55f043a1baef38862981 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Tue, 1 Nov 2022 10:10:36 -0700 Subject: [PATCH 111/173] (hopefully) final lint check. --- sdks/python/apache_beam/runners/dask/dask_runner_test.py | 2 +- sdks/python/apache_beam/runners/dask/transform_evaluator.py | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index 3ed34633107f..ca50de6d4c62 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -27,7 +27,7 @@ import dask import dask.distributed as ddist - from apache_beam.runners.dask.dask_runner import DaskOptions + from apache_beam.runners.dask.dask_runner import DaskOptions # pylint: disable=ungrouped-imports from apache_beam.runners.dask.dask_runner import DaskRunner except (ImportError, ModuleNotFoundError): raise unittest.SkipTest('Dask must be installed to run tests.') diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index eecedba24ed5..b921bc471f5c 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -29,7 +29,6 @@ import dask.bag as db import typing as t - import apache_beam from apache_beam import DoFn from apache_beam import TaggedOutput From 0fae7615c6451109c703cbb1c19e768f1e823318 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Tue, 1 Nov 2022 10:21:12 -0700 Subject: [PATCH 112/173] Disabled additional ungrouped imports check. --- sdks/python/apache_beam/runners/dask/dask_runner_test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index ca50de6d4c62..383efdd1854a 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -28,7 +28,7 @@ import dask.distributed as ddist from apache_beam.runners.dask.dask_runner import DaskOptions # pylint: disable=ungrouped-imports - from apache_beam.runners.dask.dask_runner import DaskRunner + from apache_beam.runners.dask.dask_runner import DaskRunner # pylint: disable=ungrouped-imports except (ImportError, ModuleNotFoundError): raise unittest.SkipTest('Dask must be installed to run tests.') From 119666c27f0c2a9d9b82c1cfae835ac818f84992 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Tue, 1 Nov 2022 10:23:13 -0700 Subject: [PATCH 113/173] Ran yapf with correct version. --- sdks/python/apache_beam/runners/dask/dask_runner.py | 2 -- .../apache_beam/runners/dask/dask_runner_test.py | 6 ------ sdks/python/apache_beam/runners/dask/overrides.py | 11 ----------- .../apache_beam/runners/dask/transform_evaluator.py | 7 ------- 4 files changed, 26 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index b0dff23a39ae..38bb933d191c 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -40,7 +40,6 @@ class DaskOptions(PipelineOptions): - @staticmethod def _parse_timeout(candidate): try: @@ -122,7 +121,6 @@ def metrics(self): class DaskRunner(BundleBasedDirectRunner): """Executes a pipeline on a Dask distributed client.""" - @staticmethod def to_dask_bag_visitor() -> PipelineVisitor: from dask import bag as db diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index 383efdd1854a..5ffb665e95c8 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -34,7 +34,6 @@ class DaskOptionsTest(unittest.TestCase): - def test_parses_connection_timeout__defaults_to_none(self): default_options = PipelineOptions([]) default_dask_options = default_options.view_as(DaskOptions) @@ -67,7 +66,6 @@ def test_parser_destinations__agree_with_dask_client(self): class DaskRunnerRunPipelineTest(unittest.TestCase): """Test class used to introspect the dask runner via a debugger.""" - def setUp(self) -> None: self.pipeline = test_pipeline.TestPipeline(runner=DaskRunner()) @@ -77,7 +75,6 @@ def test_create(self): assert_that(pcoll, equal_to([1])) def test_create_and_map(self): - def double(x): return x * 2 @@ -86,7 +83,6 @@ def double(x): assert_that(pcoll, equal_to([2])) def test_create_map_and_groupby(self): - def double(x): return x * 2, x @@ -95,7 +91,6 @@ def double(x): assert_that(pcoll, equal_to([(2, [1])])) def test_map_with_side_inputs(self): - def mult_by(x, y): return x * y @@ -104,7 +99,6 @@ def mult_by(x, y): assert_that(pcoll, equal_to([3])) def test_map_with_named_side_inputs(self): - def mult_by(x, y): return x * y diff --git a/sdks/python/apache_beam/runners/dask/overrides.py b/sdks/python/apache_beam/runners/dask/overrides.py index 2ff02549e41e..84f0157db5de 100644 --- a/sdks/python/apache_beam/runners/dask/overrides.py +++ b/sdks/python/apache_beam/runners/dask/overrides.py @@ -45,7 +45,6 @@ def get_windowing(self, inputs: t.Any) -> beam.Windowing: @typehints.with_input_types(K) @typehints.with_output_types(K) class _Reshuffle(beam.PTransform): - def expand(self, input_or_inputs): return beam.pvalue.PCollection.from_(input_or_inputs) @@ -61,7 +60,6 @@ def expand(self, input_or_inputs): @typehints.with_input_types(t.Tuple[K, V]) @typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) class _GroupByKeyOnly(beam.PTransform): - def expand(self, input_or_inputs): return beam.pvalue.PCollection.from_(input_or_inputs) @@ -76,7 +74,6 @@ def infer_output_type(self, input_type): @typehints.with_input_types(t.Tuple[K, t.Iterable[V]]) @typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) class _GroupAlsoByWindow(beam.ParDo): - def __init__(self, windowing): super().__init__(_GroupAlsoByWindowDoFn(windowing)) self.windowing = windowing @@ -88,7 +85,6 @@ def expand(self, input_or_inputs): @typehints.with_input_types(t.Tuple[K, V]) @typehints.with_output_types(t.Tuple[K, t.Iterable[V]]) class _GroupByKey(beam.PTransform): - def expand(self, input_or_inputs): return ( input_or_inputs @@ -98,16 +94,13 @@ def expand(self, input_or_inputs): class _Flatten(beam.PTransform): - def expand(self, input_or_inputs): is_bounded = all(pcoll.is_bounded for pcoll in input_or_inputs) return beam.pvalue.PCollection(self.pipeline, is_bounded=is_bounded) def dask_overrides() -> t.List[PTransformOverride]: - class CreateOverride(PTransformOverride): - def matches(self, applied_ptransform: AppliedPTransform) -> bool: return applied_ptransform.transform.__class__ == beam.Create @@ -116,7 +109,6 @@ def get_replacement_transform_for_applied_ptransform( return _Create(t.cast(beam.Create, applied_ptransform.transform).values) class ReshuffleOverride(PTransformOverride): - def matches(self, applied_ptransform: AppliedPTransform) -> bool: return applied_ptransform.transform.__class__ == beam.Reshuffle @@ -125,7 +117,6 @@ def get_replacement_transform_for_applied_ptransform( return _Reshuffle() class ReadOverride(PTransformOverride): - def matches(self, applied_ptransform: AppliedPTransform) -> bool: return applied_ptransform.transform.__class__ == beam.io.Read @@ -134,7 +125,6 @@ def get_replacement_transform_for_applied_ptransform( return _Read(t.cast(beam.io.Read, applied_ptransform.transform).source) class GroupByKeyOverride(PTransformOverride): - def matches(self, applied_ptransform: AppliedPTransform) -> bool: return applied_ptransform.transform.__class__ == beam.GroupByKey @@ -143,7 +133,6 @@ def get_replacement_transform_for_applied_ptransform( return _GroupByKey() class FlattenOverride(PTransformOverride): - def matches(self, applied_ptransform: AppliedPTransform) -> bool: return applied_ptransform.transform.__class__ == beam.Flatten diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index b921bc471f5c..4967e27637e2 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -104,13 +104,11 @@ def apply(self, input_bag: OpInput) -> db.Bag: class NoOp(DaskBagOp): - def apply(self, input_bag: OpInput) -> db.Bag: return input_bag class Create(DaskBagOp): - def apply(self, input_bag: OpInput) -> db.Bag: assert input_bag is None, 'Create expects no input!' original_transform = t.cast(_Create, self.transform) @@ -119,7 +117,6 @@ def apply(self, input_bag: OpInput) -> db.Bag: class ParDo(DaskBagOp): - def apply(self, input_bag: OpInput) -> db.Bag: transform = t.cast(apache_beam.ParDo, self.transform) @@ -174,7 +171,6 @@ def apply_dofn_to_bundle(items): class Map(DaskBagOp): - def apply(self, input_bag: OpInput) -> db.Bag: transform = t.cast(apache_beam.Map, self.transform) args, kwargs = util.insert_values_in_args( @@ -183,9 +179,7 @@ def apply(self, input_bag: OpInput) -> db.Bag: class GroupByKey(DaskBagOp): - def apply(self, input_bag: OpInput) -> db.Bag: - def key(item): return item[0] @@ -197,7 +191,6 @@ def value(item): class Flatten(DaskBagOp): - def apply(self, input_bag: OpInput) -> db.Bag: assert type(input_bag) is list, 'Must take a sequence of bags!' return db.concat(input_bag) From 2e46d883d28be02362c79e74f5f9e0d100820c10 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Tue, 1 Nov 2022 17:39:15 -0700 Subject: [PATCH 114/173] mini self-review. --- .../apache_beam/runners/dask/transform_evaluator.py | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 4967e27637e2..6f18fcbdca59 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -117,20 +117,18 @@ def apply(self, input_bag: OpInput) -> db.Bag: class ParDo(DaskBagOp): - def apply(self, input_bag: OpInput) -> db.Bag: + def apply(self, input_bag: db.Bag) -> db.Bag: transform = t.cast(apache_beam.ParDo, self.transform) - label = transform.label - map_fn = transform.fn args, kwargs = transform.raw_side_inputs args = list(args) main_input = next(iter(self.applied.main_inputs.values())) window_fn = main_input.windowing.windowfn if hasattr( main_input, "windowing") else None - context = DoFnContext(label, state=None) + context = DoFnContext(transform.label, state=None) bundle_finalizer_param = DoFn.BundleFinalizerParam() - do_fn_signature = DoFnSignature(map_fn) + do_fn_signature = DoFnSignature(transform.fn) tagged_receivers = OneReceiver() From 2ed8b142fb17e0b1b431549b2f7b2f1c5355e017 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Fri, 4 Nov 2022 17:03:21 -0700 Subject: [PATCH 115/173] WIP: A more correct windowing implementation with failing tests. --- .../runners/dask/transform_evaluator.py | 31 +++++++++------- sdks/python/env.yml | 35 +++++++++++++++++++ 2 files changed, 53 insertions(+), 13 deletions(-) create mode 100644 sdks/python/env.yml diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 6f18fcbdca59..955d5474c6b3 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -155,25 +155,31 @@ def apply_dofn_to_bundle(items): do_fn_invoker.invoke_setup() do_fn_invoker.invoke_start_bundle() - to_proc = list(items) + tagged_receivers.values - results = [do_fn_invoker.invoke_process(it) for it in to_proc] + for it in items: + do_fn_invoker.invoke_process(it) + + # Get results from the main receiver. + results = [v.value for v in tagged_receivers[None].values] do_fn_invoker.invoke_finish_bundle() do_fn_invoker.invoke_teardown() return results - return input_bag.map( - get_windowed_value, - window_fn).map_partitions(apply_dofn_to_bundle).flatten() + return ( + input_bag.map(get_windowed_value, window_fn) + .map_partitions(apply_dofn_to_bundle) + .flatten() + ) -class Map(DaskBagOp): - def apply(self, input_bag: OpInput) -> db.Bag: - transform = t.cast(apache_beam.Map, self.transform) - args, kwargs = util.insert_values_in_args( - transform.args, transform.kwargs, transform.side_inputs) - return input_bag.map(transform.fn.process, *args, **kwargs) +# +# class Map(DaskBagOp): +# def apply(self, input_bag: OpInput) -> db.Bag: +# transform = t.cast(apache_beam.Map, self.transform) +# args, kwargs = util.insert_values_in_args( +# transform.args, transform.kwargs, transform.side_inputs) +# return input_bag.map(transform.fn.process, *args, **kwargs) class GroupByKey(DaskBagOp): @@ -189,7 +195,7 @@ def value(item): class Flatten(DaskBagOp): - def apply(self, input_bag: OpInput) -> db.Bag: + def apply(self, input_bag: t.List[db.Bag]) -> db.Bag: assert type(input_bag) is list, 'Must take a sequence of bags!' return db.concat(input_bag) @@ -197,7 +203,6 @@ def apply(self, input_bag: OpInput) -> db.Bag: TRANSLATIONS = { _Create: Create, apache_beam.ParDo: ParDo, - apache_beam.Map: Map, _GroupByKeyOnly: GroupByKey, _Flatten: Flatten, } diff --git a/sdks/python/env.yml b/sdks/python/env.yml new file mode 100644 index 000000000000..c000b62bc5c8 --- /dev/null +++ b/sdks/python/env.yml @@ -0,0 +1,35 @@ +name: beam39 +channels: + - conda-forge +dependencies: + - python=3.9 + - protobuf>3.12.2,<3.19.5 + - crcmod>=1.7,<2.0 + - orjson<4.0 + - dill>=0.3.1.1,<0.3.2 + - cloudpickle~=2.2.0 + - fastavro>=0.23.6,<2 + - fasteners>=0.3,<1.0 + - grpcio>=1.33.1,!=1.48.0,<2 + - httplib2>=0.8,<0.21.0 + - numpy>=1.14.3,<1.23.0 + - pymongo>=3.8.0,<4.0.0 + - proto-plus>=1.7.1,<2 + - pydot>=1.2.0,<2 + - python-dateutil>=2.8.0,<3 + - pytz>=2018.3 + - regex>=2020.6.8 + - requests>=2.24.0,<3.0.0 + - typing-extensions>=3.7.0 + - zstandard>=0.18.0,<1 + - pyarrow>=0.15.1,<10.0.0 + - dask>=2022.6 + - distributed>=2022.6 + - mypy-protobuf==3.3.0 + - python-hdfs + - pytest + - pip + - pip: + - hdfs>=2.1.0,<3.0.0 + - objsize>=0.5.2,<0.6.0 + From 2f193d52b89adc09c38cf20585c963c46318a507 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Fri, 4 Nov 2022 17:42:22 -0700 Subject: [PATCH 116/173] WIP: Further improvements, more correct windowing impl. The error that I'm hitting seems to be in creating tuples during the CoGroupByKey. --- .../apache_beam/runners/dask/transform_evaluator.py | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 955d5474c6b3..ab1054419697 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -158,20 +158,21 @@ def apply_dofn_to_bundle(items): for it in items: do_fn_invoker.invoke_process(it) - # Get results from the main receiver. - results = [v.value for v in tagged_receivers[None].values] + results = [v.value for v in tagged_receivers.values] do_fn_invoker.invoke_finish_bundle() do_fn_invoker.invoke_teardown() return results - return ( - input_bag.map(get_windowed_value, window_fn) + x = ( + input_bag + .map(get_windowed_value, window_fn) .map_partitions(apply_dofn_to_bundle) - .flatten() ) + return x + # # class Map(DaskBagOp): From 577f30a15ac370e7b5052a7566c224193c5cac72 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Fri, 4 Nov 2022 23:41:38 -0700 Subject: [PATCH 117/173] WIP: Passing initial tests, failing multiple -- drops random elements during groupbykey. --- .../runners/dask/dask_runner_test.py | 21 ++++++++++++++++ .../runners/dask/transform_evaluator.py | 25 ++++++++++--------- 2 files changed, 34 insertions(+), 12 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index 5ffb665e95c8..903f6a2880f5 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -74,6 +74,11 @@ def test_create(self): pcoll = p | beam.Create([1]) assert_that(pcoll, equal_to([1])) + def test_create_multiple(self): + with self.pipeline as p: + pcoll = p | beam.Create([1, 2, 3]) + assert_that(pcoll, equal_to([1, 2, 3])) + def test_create_and_map(self): def double(x): return x * 2 @@ -82,6 +87,14 @@ def double(x): pcoll = p | beam.Create([1]) | beam.Map(double) assert_that(pcoll, equal_to([2])) + def test_create_and_map_multiple(self): + def double(x): + return x * 2 + + with self.pipeline as p: + pcoll = p | beam.Create([1, 2]) | beam.Map(double) + assert_that(pcoll, equal_to([2, 4])) + def test_create_map_and_groupby(self): def double(x): return x * 2, x @@ -90,6 +103,14 @@ def double(x): pcoll = p | beam.Create([1]) | beam.Map(double) | beam.GroupByKey() assert_that(pcoll, equal_to([(2, [1])])) + def test_create_map_and_groupby_multiple(self): + def double(x): + return x * 2, x + + with self.pipeline as p: + pcoll = p | beam.Create([1, 2, 1, 2, 3]) | beam.Map(double) | beam.GroupByKey() + assert_that(pcoll, equal_to([(2, [1, 1]), (4, [2, 2]), (3, [6])])) + def test_map_with_side_inputs(self): def mult_by(x, y): return x * y diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index ab1054419697..a1bb56e49f5d 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -67,6 +67,12 @@ def get_windowed_value(item: t.Any, window_fn: WindowFn) -> WindowedValue: return windowed_value +def defenestrate(x): + if isinstance(x, WindowedValue): + return x.value + return x + + @dataclasses.dataclass class TaggingReceiver(Receiver): tag: str @@ -174,31 +180,26 @@ def apply_dofn_to_bundle(items): return x -# -# class Map(DaskBagOp): -# def apply(self, input_bag: OpInput) -> db.Bag: -# transform = t.cast(apache_beam.Map, self.transform) -# args, kwargs = util.insert_values_in_args( -# transform.args, transform.kwargs, transform.side_inputs) -# return input_bag.map(transform.fn.process, *args, **kwargs) - - class GroupByKey(DaskBagOp): def apply(self, input_bag: OpInput) -> db.Bag: + def key(item): return item[0] def value(item): k, v = item - return k, [elm[1] for elm in v] + return k, [defenestrate(elm[1]) for elm in v] + + y = input_bag.groupby(key).map(value) - return input_bag.groupby(key).map(value) + return y class Flatten(DaskBagOp): def apply(self, input_bag: t.List[db.Bag]) -> db.Bag: assert type(input_bag) is list, 'Must take a sequence of bags!' - return db.concat(input_bag) + c = db.concat(input_bag) + return c TRANSLATIONS = { From b3a70f674b0383b800244cb271c5306fcbed74d8 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 5 Nov 2022 00:01:52 -0700 Subject: [PATCH 118/173] WIP: All tests are passing :) out-of-order issue was fixed by calling dask.optimize. --- sdks/python/apache_beam/runners/dask/dask_runner.py | 5 ++++- .../apache_beam/runners/dask/dask_runner_test.py | 10 +++++++++- .../apache_beam/runners/dask/transform_evaluator.py | 3 ++- 3 files changed, 15 insertions(+), 3 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index 38bb933d191c..8e8cbb60e056 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -26,6 +26,8 @@ import argparse import typing as t +import dask + from apache_beam import pvalue from apache_beam.options.pipeline_options import PipelineOptions from apache_beam.pipeline import AppliedPTransform @@ -179,5 +181,6 @@ def run_pipeline(self, pipeline, options): dask_visitor = self.to_dask_bag_visitor() pipeline.visit(dask_visitor) - futures = client.compute(list(dask_visitor.bags.values())) + opt = dask.optimize(dask_visitor.bags.values()) + futures = client.compute(opt) return DaskRunnerResult(client, futures) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index 903f6a2880f5..161247cb84a8 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -95,6 +95,14 @@ def double(x): pcoll = p | beam.Create([1, 2]) | beam.Map(double) assert_that(pcoll, equal_to([2, 4])) + def test_create_and_map_many(self): + def double(x): + return x * 2 + + with self.pipeline as p: + pcoll = p | beam.Create(list(range(1, 11))) | beam.Map(double) + assert_that(pcoll, equal_to(list(range(2, 21, 2)))) + def test_create_map_and_groupby(self): def double(x): return x * 2, x @@ -109,7 +117,7 @@ def double(x): with self.pipeline as p: pcoll = p | beam.Create([1, 2, 1, 2, 3]) | beam.Map(double) | beam.GroupByKey() - assert_that(pcoll, equal_to([(2, [1, 1]), (4, [2, 2]), (3, [6])])) + assert_that(pcoll, equal_to([(2, [1, 1]), (4, [2, 2]), (6, [3])])) def test_map_with_side_inputs(self): def mult_by(x, y): diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index a1bb56e49f5d..517bc620fe75 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -188,7 +188,8 @@ def key(item): def value(item): k, v = item - return k, [defenestrate(elm[1]) for elm in v] + d = [defenestrate(elm[1]) for elm in v] + return k, d y = input_bag.groupby(key).map(value) From 7acd8d5f5fbc1c1474792d3df43d22e0a15d90db Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 5 Nov 2022 00:03:58 -0700 Subject: [PATCH 119/173] Cleanup: removed variables for debugger. --- .../python/apache_beam/runners/dask/dask_runner.py | 4 ++-- .../runners/dask/transform_evaluator.py | 14 ++++---------- 2 files changed, 6 insertions(+), 12 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index 8e8cbb60e056..5f055a344ebc 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -181,6 +181,6 @@ def run_pipeline(self, pipeline, options): dask_visitor = self.to_dask_bag_visitor() pipeline.visit(dask_visitor) - opt = dask.optimize(dask_visitor.bags.values()) - futures = client.compute(opt) + opt_graph = dask.optimize(dask_visitor.bags.values()) + futures = client.compute(opt_graph) return DaskRunnerResult(client, futures) diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 517bc620fe75..1a4672db9ebf 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -171,14 +171,12 @@ def apply_dofn_to_bundle(items): return results - x = ( + return ( input_bag .map(get_windowed_value, window_fn) .map_partitions(apply_dofn_to_bundle) ) - return x - class GroupByKey(DaskBagOp): def apply(self, input_bag: OpInput) -> db.Bag: @@ -188,19 +186,15 @@ def key(item): def value(item): k, v = item - d = [defenestrate(elm[1]) for elm in v] - return k, d - - y = input_bag.groupby(key).map(value) + return k, [defenestrate(elm[1]) for elm in v] - return y + return input_bag.groupby(key).map(value) class Flatten(DaskBagOp): def apply(self, input_bag: t.List[db.Bag]) -> db.Bag: assert type(input_bag) is list, 'Must take a sequence of bags!' - c = db.concat(input_bag) - return c + return db.concat(input_bag) TRANSLATIONS = { From 7e90e2bf6b206eede9ba828127b05bbcf32f9da3 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 5 Nov 2022 00:08:29 -0700 Subject: [PATCH 120/173] Lint + YAPF --- sdks/python/apache_beam/runners/dask/dask_runner.py | 4 ++-- sdks/python/apache_beam/runners/dask/dask_runner_test.py | 3 ++- .../apache_beam/runners/dask/transform_evaluator.py | 8 ++------ 3 files changed, 6 insertions(+), 9 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index 5f055a344ebc..9f5c5bb9b97f 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -26,8 +26,6 @@ import argparse import typing as t -import dask - from apache_beam import pvalue from apache_beam.options.pipeline_options import PipelineOptions from apache_beam.pipeline import AppliedPTransform @@ -162,6 +160,8 @@ def is_fnapi_compatible(): return False def run_pipeline(self, pipeline, options): + import dask + # TODO(alxr): Create interactive notebook support. if is_in_notebook(): raise NotImplementedError('interactive support will come later!') diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index 161247cb84a8..1e26babbb791 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -116,7 +116,8 @@ def double(x): return x * 2, x with self.pipeline as p: - pcoll = p | beam.Create([1, 2, 1, 2, 3]) | beam.Map(double) | beam.GroupByKey() + pcoll = p | beam.Create([1, 2, 1, 2, 3 + ]) | beam.Map(double) | beam.GroupByKey() assert_that(pcoll, equal_to([(2, [1, 1]), (4, [2, 2]), (6, [3])])) def test_map_with_side_inputs(self): diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 1a4672db9ebf..510423950c74 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -32,7 +32,6 @@ import apache_beam from apache_beam import DoFn from apache_beam import TaggedOutput -from apache_beam.internal import util from apache_beam.pipeline import AppliedPTransform from apache_beam.runners.common import DoFnContext from apache_beam.runners.common import DoFnInvoker @@ -172,15 +171,12 @@ def apply_dofn_to_bundle(items): return results return ( - input_bag - .map(get_windowed_value, window_fn) - .map_partitions(apply_dofn_to_bundle) - ) + input_bag.map(get_windowed_value, + window_fn).map_partitions(apply_dofn_to_bundle)) class GroupByKey(DaskBagOp): def apply(self, input_bag: OpInput) -> db.Bag: - def key(item): return item[0] From 6e33ce28ccdedf0b5b6e875856e2602fde9eacba Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 5 Nov 2022 00:12:30 -0700 Subject: [PATCH 121/173] self-review. --- .../runners/dask/transform_evaluator.py | 7 ++-- sdks/python/env.yml | 35 ------------------- 2 files changed, 3 insertions(+), 39 deletions(-) delete mode 100644 sdks/python/env.yml diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 510423950c74..8bcc61ce7da7 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -170,13 +170,12 @@ def apply_dofn_to_bundle(items): return results - return ( - input_bag.map(get_windowed_value, - window_fn).map_partitions(apply_dofn_to_bundle)) + return input_bag.map(get_windowed_value, + window_fn).map_partitions(apply_dofn_to_bundle) class GroupByKey(DaskBagOp): - def apply(self, input_bag: OpInput) -> db.Bag: + def apply(self, input_bag: db.Bag) -> db.Bag: def key(item): return item[0] diff --git a/sdks/python/env.yml b/sdks/python/env.yml deleted file mode 100644 index c000b62bc5c8..000000000000 --- a/sdks/python/env.yml +++ /dev/null @@ -1,35 +0,0 @@ -name: beam39 -channels: - - conda-forge -dependencies: - - python=3.9 - - protobuf>3.12.2,<3.19.5 - - crcmod>=1.7,<2.0 - - orjson<4.0 - - dill>=0.3.1.1,<0.3.2 - - cloudpickle~=2.2.0 - - fastavro>=0.23.6,<2 - - fasteners>=0.3,<1.0 - - grpcio>=1.33.1,!=1.48.0,<2 - - httplib2>=0.8,<0.21.0 - - numpy>=1.14.3,<1.23.0 - - pymongo>=3.8.0,<4.0.0 - - proto-plus>=1.7.1,<2 - - pydot>=1.2.0,<2 - - python-dateutil>=2.8.0,<3 - - pytz>=2018.3 - - regex>=2020.6.8 - - requests>=2.24.0,<3.0.0 - - typing-extensions>=3.7.0 - - zstandard>=0.18.0,<1 - - pyarrow>=0.15.1,<10.0.0 - - dask>=2022.6 - - distributed>=2022.6 - - mypy-protobuf==3.3.0 - - python-hdfs - - pytest - - pip - - pip: - - hdfs>=2.1.0,<3.0.0 - - objsize>=0.5.2,<0.6.0 - From f54f14c9f4f6c3adc2d50fe627fcc16631e759c2 Mon Sep 17 00:00:00 2001 From: Pablo E Date: Fri, 18 Nov 2022 14:47:58 -0800 Subject: [PATCH 122/173] fix lint --- sdks/python/apache_beam/runners/dask/dask_runner.py | 1 - sdks/python/apache_beam/runners/dask/overrides.py | 1 - .../python/apache_beam/runners/dask/transform_evaluator.py | 7 +++---- 3 files changed, 3 insertions(+), 6 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index 9f5c5bb9b97f..513155fd1659 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -22,7 +22,6 @@ scheduler. """ import dataclasses - import argparse import typing as t diff --git a/sdks/python/apache_beam/runners/dask/overrides.py b/sdks/python/apache_beam/runners/dask/overrides.py index 84f0157db5de..46cb14cad870 100644 --- a/sdks/python/apache_beam/runners/dask/overrides.py +++ b/sdks/python/apache_beam/runners/dask/overrides.py @@ -15,7 +15,6 @@ # limitations under the License. # import dataclasses - import typing as t import apache_beam as beam diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 8bcc61ce7da7..02aaffd0658a 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -22,14 +22,13 @@ TODO(alxr): Translate ops from https://docs.dask.org/en/latest/bag-api.html. """ -import dataclasses -from dataclasses import field - import abc -import dask.bag as db +import dataclasses import typing as t +from dataclasses import field import apache_beam +import dask.bag as db from apache_beam import DoFn from apache_beam import TaggedOutput from apache_beam.pipeline import AppliedPTransform From 8dd2cdba75ee08ec0d6dd292b771031cd67a6aae Mon Sep 17 00:00:00 2001 From: Pablo E Date: Fri, 18 Nov 2022 15:59:01 -0800 Subject: [PATCH 123/173] fixup --- sdks/python/apache_beam/runners/dask/dask_runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index 513155fd1659..0375e464d23b 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -21,8 +21,8 @@ transformations across processes and workers via Dask distributed's scheduler. """ -import dataclasses import argparse +import dataclasses import typing as t from apache_beam import pvalue From 518a8f0a7ad66ce6986ffaceeec5424cc3874293 Mon Sep 17 00:00:00 2001 From: Pablo E Date: Sat, 19 Nov 2022 11:33:11 -0800 Subject: [PATCH 124/173] ignore internal dask file for docs --- sdks/python/scripts/generate_pydoc.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/sdks/python/scripts/generate_pydoc.sh b/sdks/python/scripts/generate_pydoc.sh index ae715bb3a026..5d15bfae2997 100755 --- a/sdks/python/scripts/generate_pydoc.sh +++ b/sdks/python/scripts/generate_pydoc.sh @@ -64,6 +64,7 @@ excluded_patterns=( 'apache_beam/runners/portability/' 'apache_beam/runners/test/' 'apache_beam/runners/worker/' + 'apache_beam/runners/dask/transform_evaluator.*' 'apache_beam/testing/benchmarks/chicago_taxi/' 'apache_beam/testing/benchmarks/inference/' 'apache_beam/testing/benchmarks/data/' From 801b1316e31603f063f83db03058bf0ee8b28b13 Mon Sep 17 00:00:00 2001 From: Pablo E Date: Mon, 28 Nov 2022 13:30:16 -0800 Subject: [PATCH 125/173] fixing dask version --- sdks/python/setup.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/python/setup.py b/sdks/python/setup.py index 61858fa5d978..ff9158b747b7 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -352,8 +352,8 @@ def get_portability_package_data(): 'dataframe': ['pandas>=1.0,<1.5;python_version<"3.10"', 'pandas>=1.4.3,<1.5;python_version>="3.10"'], 'dask': [ - 'dask >= 2022.6', - 'distributed >= 2022.6', + 'dask >= 2022.6.0', + 'distributed >= 2022.6.0', ], }, zip_safe=False, From f89f609fdf9a0afdbb5fc61b2dd938c762a14d30 Mon Sep 17 00:00:00 2001 From: Pablo E Date: Mon, 28 Nov 2022 13:45:47 -0800 Subject: [PATCH 126/173] remove Python 3.7 which seems unsupported by newer Dask versions --- .github/workflows/dask_runner_tests.yml | 1 - 1 file changed, 1 deletion(-) diff --git a/.github/workflows/dask_runner_tests.yml b/.github/workflows/dask_runner_tests.yml index 33d0575e2c8c..3d9165b63a97 100644 --- a/.github/workflows/dask_runner_tests.yml +++ b/.github/workflows/dask_runner_tests.yml @@ -67,7 +67,6 @@ jobs: matrix: os: [ubuntu-latest, macos-latest, windows-latest] params: [ - {"py_ver": "3.7", "tox_env": "py37"}, {"py_ver": "3.8", "tox_env": "py38"}, {"py_ver": "3.9", "tox_env": "py39"}, {"py_ver": "3.10", "tox_env": "py310" }, From d6486fe1e4be931b40f27012e0848df9048f1be0 Mon Sep 17 00:00:00 2001 From: Pablo E Date: Tue, 29 Nov 2022 18:30:12 -0800 Subject: [PATCH 127/173] reducing scope of Dask tests --- .github/workflows/dask_runner_tests.yml | 8 ++------ sdks/python/tox.ini | 1 + 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/.github/workflows/dask_runner_tests.yml b/.github/workflows/dask_runner_tests.yml index 3d9165b63a97..4f2e9606948b 100644 --- a/.github/workflows/dask_runner_tests.yml +++ b/.github/workflows/dask_runner_tests.yml @@ -65,7 +65,7 @@ jobs: strategy: fail-fast: false matrix: - os: [ubuntu-latest, macos-latest, windows-latest] + os: [ubuntu-latest, macos-latest] params: [ {"py_ver": "3.8", "tox_env": "py38"}, {"py_ver": "3.9", "tox_env": "py39"}, @@ -85,15 +85,11 @@ jobs: run: pip install tox - name: Install SDK with dask working-directory: ./sdks/python - run: pip install setuptools --upgrade && pip install -e .[gcp,dask,test] + run: pip install setuptools --upgrade && pip install -e .[dask,test] - name: Run tests basic unix if: startsWith(matrix.os, 'ubuntu') || startsWith(matrix.os, 'macos') working-directory: ./sdks/python run: tox -c tox.ini -e ${{ matrix.params.tox_env }}-dask - - name: Run tests basic windows - if: startsWith(matrix.os, 'windows') - working-directory: ./sdks/python - run: tox -c tox.ini -e ${{ matrix.params.tox_env }}-win-dask - name: Upload test logs uses: actions/upload-artifact@v3 if: always() diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini index b936a6a4de84..b0a60a8057cd 100644 --- a/sdks/python/tox.ini +++ b/sdks/python/tox.ini @@ -96,6 +96,7 @@ commands = extras = test,dask commands = {toxinidir}/scripts/run_pytest.sh {envname} "{posargs}" + [testenv:py38-cloudcoverage] deps = codecov From 6f05963ab28c11eeb5b272890e076d54aba31fad Mon Sep 17 00:00:00 2001 From: Pablo E Date: Tue, 29 Nov 2022 21:16:34 -0800 Subject: [PATCH 128/173] adding datafrems dep --- .github/workflows/dask_runner_tests.yml | 2 +- sdks/python/tox.ini | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/dask_runner_tests.yml b/.github/workflows/dask_runner_tests.yml index 4f2e9606948b..1a4d5906916d 100644 --- a/.github/workflows/dask_runner_tests.yml +++ b/.github/workflows/dask_runner_tests.yml @@ -85,7 +85,7 @@ jobs: run: pip install tox - name: Install SDK with dask working-directory: ./sdks/python - run: pip install setuptools --upgrade && pip install -e .[dask,test] + run: pip install setuptools --upgrade && pip install -e .[dask,test,dataframes] - name: Run tests basic unix if: startsWith(matrix.os, 'ubuntu') || startsWith(matrix.os, 'macos') working-directory: ./sdks/python diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini index b0a60a8057cd..d09c00fdf0ed 100644 --- a/sdks/python/tox.ini +++ b/sdks/python/tox.ini @@ -93,7 +93,7 @@ commands = {toxinidir}/scripts/run_pytest.sh {envname} "{posargs}" [testenv:py{37,38,39}-dask] -extras = test,dask +extras = test,dask,dataframes commands = {toxinidir}/scripts/run_pytest.sh {envname} "{posargs}" From 8b40a340cf97dcc8aaa1bf1f3443c9944c9a534f Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 22 Jul 2023 18:40:00 -0700 Subject: [PATCH 129/173] Simpler wait to implement `wait_until_finish()` (Dask offers quite an elegant API). --- sdks/python/apache_beam/runners/dask/dask_runner.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index 0375e464d23b..ed9b76017a4d 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -95,12 +95,12 @@ def __post_init__(self): super().__init__(PipelineState.RUNNING) def wait_until_finish(self, duration=None) -> str: + from dask import distributed try: if duration is not None: # Convert milliseconds to seconds duration /= 1000 - self.client.wait_for_workers(timeout=duration) - self.client.gather(self.futures, errors='raise') + distributed.wait(self.futures, timeout=duration) self._state = PipelineState.DONE except: # pylint: disable=broad-except self._state = PipelineState.FAILED From cae8561399a9c2587fa8d26a1f587c0a2f7e2895 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 22 Jul 2023 19:34:15 -0700 Subject: [PATCH 130/173] Added a test for fixed windowing. --- .../runners/dask/dask_runner_test.py | 23 +++++++++++++++++++ 1 file changed, 23 insertions(+) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index 1e26babbb791..b3aa86fb09a1 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -16,12 +16,14 @@ # import inspect import unittest +import datetime import apache_beam as beam from apache_beam.options.pipeline_options import PipelineOptions from apache_beam.testing import test_pipeline from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to +from apache_beam.transforms import window try: import dask @@ -136,6 +138,27 @@ def mult_by(x, y): pcoll = p | beam.Create([1]) | beam.Map(mult_by, y=3) assert_that(pcoll, equal_to([3])) + def test_groupby_with_fixed_windows(self): + def double(x): + return x * 2, x + + def add_timestamp(pair): + now = ( + datetime.datetime.now() + datetime.timedelta(seconds=pair[1] * 60) + ).timestamp() + return window.TimestampedValue(pair, now) + + with self.pipeline as p: + pcoll = ( + p + | beam.Create([1, 2, 1, 2, 3]) + | beam.Map(double) + | beam.WindowInto(window.FixedWindows(60)) + | beam.Map(add_timestamp) + | beam.GroupByKey() + ) + assert_that(pcoll, equal_to([(2, [1, 1]), (4, [2, 2]), (6, [3])])) + if __name__ == '__main__': unittest.main() From c82a2415d3867d5ad37c416e8e3707e0d3f30b54 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 22 Jul 2023 21:35:11 -0700 Subject: [PATCH 131/173] Attempt to fix lint and threading error. --- sdks/python/apache_beam/runners/dask/dask_runner.py | 9 ++++++--- sdks/python/apache_beam/runners/dask/dask_runner_test.py | 7 +++---- 2 files changed, 9 insertions(+), 7 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index ed9b76017a4d..8f14de042a8c 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -36,6 +36,12 @@ from apache_beam.runners.runner import PipelineResult from apache_beam.runners.runner import PipelineState from apache_beam.utils.interactive_utils import is_in_notebook +try: + # Added to try to prevent threading related issues, see + # https://github.com/pytest-dev/pytest/issues/3216#issuecomment-1502451456 + from dask import distributed +except ImportError: + distributed = {} class DaskOptions(PipelineOptions): @@ -86,8 +92,6 @@ def _add_argparse_args(cls, parser: argparse.ArgumentParser) -> None: @dataclasses.dataclass class DaskRunnerResult(PipelineResult): - from dask import distributed - client: distributed.Client futures: t.Sequence[distributed.Future] @@ -95,7 +99,6 @@ def __post_init__(self): super().__init__(PipelineState.RUNNING) def wait_until_finish(self, duration=None) -> str: - from dask import distributed try: if duration is not None: # Convert milliseconds to seconds diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index b3aa86fb09a1..7526273794ad 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -14,9 +14,9 @@ # See the License for the specific language governing permissions and # limitations under the License. # +import datetime import inspect import unittest -import datetime import apache_beam as beam from apache_beam.options.pipeline_options import PipelineOptions @@ -143,9 +143,8 @@ def double(x): return x * 2, x def add_timestamp(pair): - now = ( - datetime.datetime.now() + datetime.timedelta(seconds=pair[1] * 60) - ).timestamp() + delta = datetime.timedelta(seconds=pair[1] * 60) + now = (datetime.datetime.now() + delta).timestamp() return window.TimestampedValue(pair, now) with self.pipeline as p: From ad43ebe136dff5120c0ed17491f27ab5d3343902 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Sat, 22 Jul 2023 22:24:59 -0700 Subject: [PATCH 132/173] More lint fixing. --- sdks/python/apache_beam/runners/dask/dask_runner.py | 1 + sdks/python/apache_beam/runners/dask/dask_runner_test.py | 3 +-- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index 8f14de042a8c..f11d139c0c94 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -36,6 +36,7 @@ from apache_beam.runners.runner import PipelineResult from apache_beam.runners.runner import PipelineState from apache_beam.utils.interactive_utils import is_in_notebook + try: # Added to try to prevent threading related issues, see # https://github.com/pytest-dev/pytest/issues/3216#issuecomment-1502451456 diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index 7526273794ad..6f642cc2f010 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -154,8 +154,7 @@ def add_timestamp(pair): | beam.Map(double) | beam.WindowInto(window.FixedWindows(60)) | beam.Map(add_timestamp) - | beam.GroupByKey() - ) + | beam.GroupByKey()) assert_that(pcoll, equal_to([(2, [1, 1]), (4, [2, 2]), (6, [3])])) From 71c10d977c1bc8333f2174381c1a7d88faef1dc7 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Mon, 31 Jul 2023 14:16:28 -0700 Subject: [PATCH 133/173] Added docstrings. --- .../runners/dask/transform_evaluator.py | 24 +++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 02aaffd0658a..2b6586f8f13e 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -45,11 +45,15 @@ from apache_beam.transforms.window import WindowFn from apache_beam.utils.windowed_value import WindowedValue +"""Inputs to DaskOps.""" OpInput = t.Union[db.Bag, t.Sequence[db.Bag], None] + +"""Value types for PCollections (possibly Windowed Values).""" PCollVal = t.Union[WindowedValue, t.Any] def get_windowed_value(item: t.Any, window_fn: WindowFn) -> WindowedValue: + """Wraps a value (item) inside a Window.""" if isinstance(item, TaggedOutput): item = item.value @@ -66,6 +70,7 @@ def get_windowed_value(item: t.Any, window_fn: WindowFn) -> WindowedValue: def defenestrate(x): + """Extracts the underlying item from a Window.""" if isinstance(x, WindowedValue): return x.value return x @@ -73,6 +78,7 @@ def defenestrate(x): @dataclasses.dataclass class TaggingReceiver(Receiver): + """A Receiver that handles tagged `WindowValue`s.""" tag: str values: t.List[PCollVal] @@ -86,6 +92,7 @@ def receive(self, windowed_value: WindowedValue): @dataclasses.dataclass class OneReceiver(dict): + """A Receiver that tags value via dictionary lookup key.""" values: t.List[PCollVal] = field(default_factory=list) def __missing__(self, key): @@ -96,6 +103,15 @@ def __missing__(self, key): @dataclasses.dataclass class DaskBagOp(abc.ABC): + """Abstract Base Class for all Dask-supported Operations. + + All DaskBagOps must support an `apply()` operation, which invokes the dask + bag upon the previous op's input. + + Attributes + applied: The underlying `AppliedPTransform` which holds the code for the + target operation. + """ applied: AppliedPTransform @property @@ -108,11 +124,13 @@ def apply(self, input_bag: OpInput) -> db.Bag: class NoOp(DaskBagOp): + """An identity on a dask bag: returns the input as-is.""" def apply(self, input_bag: OpInput) -> db.Bag: return input_bag class Create(DaskBagOp): + """The beginning of a Beam pipeline; the input must be `None`.""" def apply(self, input_bag: OpInput) -> db.Bag: assert input_bag is None, 'Create expects no input!' original_transform = t.cast(_Create, self.transform) @@ -121,6 +139,10 @@ def apply(self, input_bag: OpInput) -> db.Bag: class ParDo(DaskBagOp): + """Apply a pure function in an embarrassingly-parallel way. + + This consumes a sequence of items and returns a sequence of items. + """ def apply(self, input_bag: db.Bag) -> db.Bag: transform = t.cast(apache_beam.ParDo, self.transform) @@ -174,6 +196,7 @@ def apply_dofn_to_bundle(items): class GroupByKey(DaskBagOp): + """Group a PCollection into a mapping of keys to elements.""" def apply(self, input_bag: db.Bag) -> db.Bag: def key(item): return item[0] @@ -186,6 +209,7 @@ def value(item): class Flatten(DaskBagOp): + """Produces a flattened bag from a collection of bags.""" def apply(self, input_bag: t.List[db.Bag]) -> db.Bag: assert type(input_bag) is list, 'Must take a sequence of bags!' return db.concat(input_bag) From ff5f47c56c2e2a0e0026cebea22a9ccb4504d4bf Mon Sep 17 00:00:00 2001 From: Charles Stern <62192187+cisaacstern@users.noreply.github.com> Date: Thu, 5 Oct 2023 17:25:50 -0400 Subject: [PATCH 134/173] can @cisaacstern commit to this branch? From a83e85a9891bb602979c36342497d3b89213c89e Mon Sep 17 00:00:00 2001 From: Charles Stern <62192187+cisaacstern@users.noreply.github.com> Date: Wed, 11 Oct 2023 16:46:49 -0700 Subject: [PATCH 135/173] ran yapf manually, per beam dev wiki --- sdks/python/apache_beam/runners/dask/transform_evaluator.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 2b6586f8f13e..139f6f8e9b7d 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -44,10 +44,8 @@ from apache_beam.transforms.window import TimestampedValue from apache_beam.transforms.window import WindowFn from apache_beam.utils.windowed_value import WindowedValue - """Inputs to DaskOps.""" OpInput = t.Union[db.Bag, t.Sequence[db.Bag], None] - """Value types for PCollections (possibly Windowed Values).""" PCollVal = t.Union[WindowedValue, t.Any] From dea3e112baf805423bcf5757ec4d3d6fb4b826bc Mon Sep 17 00:00:00 2001 From: Charles Stern <62192187+cisaacstern@users.noreply.github.com> Date: Wed, 11 Oct 2023 17:09:28 -0700 Subject: [PATCH 136/173] for pylint: describe types with comments, not literals --- sdks/python/apache_beam/runners/dask/transform_evaluator.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 139f6f8e9b7d..82e62ccfa1f3 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -44,9 +44,11 @@ from apache_beam.transforms.window import TimestampedValue from apache_beam.transforms.window import WindowFn from apache_beam.utils.windowed_value import WindowedValue -"""Inputs to DaskOps.""" + +# Inputs to DaskOps. OpInput = t.Union[db.Bag, t.Sequence[db.Bag], None] -"""Value types for PCollections (possibly Windowed Values).""" + +# Value types for PCollections (possibly Windowed Values). PCollVal = t.Union[WindowedValue, t.Any] From 32ee6e13b1d1853b5760ef480d76da581ddadb79 Mon Sep 17 00:00:00 2001 From: Charles Stern <62192187+cisaacstern@users.noreply.github.com> Date: Thu, 12 Oct 2023 16:41:50 -0700 Subject: [PATCH 137/173] use isinstance in Flatten.apply --- sdks/python/apache_beam/runners/dask/transform_evaluator.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 82e62ccfa1f3..9a19da59589b 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -211,7 +211,7 @@ def value(item): class Flatten(DaskBagOp): """Produces a flattened bag from a collection of bags.""" def apply(self, input_bag: t.List[db.Bag]) -> db.Bag: - assert type(input_bag) is list, 'Must take a sequence of bags!' + assert isinstance(input_bag, list), 'Must take a sequence of bags!' return db.concat(input_bag) From 305699bbbdde9b555ecd88c4bedc0b312a268a53 Mon Sep 17 00:00:00 2001 From: Charles Stern <62192187+cisaacstern@users.noreply.github.com> Date: Thu, 12 Oct 2023 16:42:52 -0700 Subject: [PATCH 138/173] cast dask_visitor.bags.values() to list in dask.optimize call --- sdks/python/apache_beam/runners/dask/dask_runner.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index f11d139c0c94..81e263d82626 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -183,7 +183,6 @@ def run_pipeline(self, pipeline, options): dask_visitor = self.to_dask_bag_visitor() pipeline.visit(dask_visitor) - - opt_graph = dask.optimize(dask_visitor.bags.values()) + opt_graph = dask.optimize(list(dask_visitor.bags.values())) futures = client.compute(opt_graph) return DaskRunnerResult(client, futures) From 9557e3aba4b67ca85d770bcfd9cc677affce2178 Mon Sep 17 00:00:00 2001 From: Charles Stern <62192187+cisaacstern@users.noreply.github.com> Date: Tue, 17 Oct 2023 15:14:40 -0700 Subject: [PATCH 139/173] revert casting bag vals to list --- sdks/python/apache_beam/runners/dask/dask_runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index 81e263d82626..341f383011df 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -183,6 +183,6 @@ def run_pipeline(self, pipeline, options): dask_visitor = self.to_dask_bag_visitor() pipeline.visit(dask_visitor) - opt_graph = dask.optimize(list(dask_visitor.bags.values())) + opt_graph = dask.optimize(dask_visitor.bags.values()) futures = client.compute(opt_graph) return DaskRunnerResult(client, futures) From 899fe083599ffc8d96588f6d68fa442268da7ee3 Mon Sep 17 00:00:00 2001 From: Charles Stern <62192187+cisaacstern@users.noreply.github.com> Date: Tue, 24 Oct 2023 16:15:09 -0700 Subject: [PATCH 140/173] cast and unpack list --- sdks/python/apache_beam/runners/dask/dask_runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index 341f383011df..49ba405d2838 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -183,6 +183,6 @@ def run_pipeline(self, pipeline, options): dask_visitor = self.to_dask_bag_visitor() pipeline.visit(dask_visitor) - opt_graph = dask.optimize(dask_visitor.bags.values()) + opt_graph = dask.optimize(*list(dask_visitor.bags.values())) futures = client.compute(opt_graph) return DaskRunnerResult(client, futures) From c398a21c83518480b4b2572304222fd2ec3f6499 Mon Sep 17 00:00:00 2001 From: Charles Stern <62192187+cisaacstern@users.noreply.github.com> Date: Thu, 26 Oct 2023 12:58:13 -0700 Subject: [PATCH 141/173] for serialization, move apply_do_fn_to_bundle to module level --- .../runners/dask/transform_evaluator.py | 83 ++++++++++--------- 1 file changed, 45 insertions(+), 38 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 9a19da59589b..be10cceba1be 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -138,6 +138,26 @@ def apply(self, input_bag: OpInput) -> db.Bag: return db.from_sequence(items) +def apply_dofn_to_bundle( + items, do_fn_invoker_args, do_fn_invoker_kwargs, tagged_receivers): + + do_fn_invoker = DoFnInvoker.create_invoker( + *do_fn_invoker_args, **do_fn_invoker_kwargs) + + do_fn_invoker.invoke_setup() + do_fn_invoker.invoke_start_bundle() + + for it in items: + do_fn_invoker.invoke_process(it) + + results = [v.value for v in tagged_receivers.values] + + do_fn_invoker.invoke_finish_bundle() + do_fn_invoker.invoke_teardown() + + return results + + class ParDo(DaskBagOp): """Apply a pure function in an embarrassingly-parallel way. @@ -152,47 +172,34 @@ def apply(self, input_bag: db.Bag) -> db.Bag: window_fn = main_input.windowing.windowfn if hasattr( main_input, "windowing") else None - context = DoFnContext(transform.label, state=None) - bundle_finalizer_param = DoFn.BundleFinalizerParam() - do_fn_signature = DoFnSignature(transform.fn) - tagged_receivers = OneReceiver() - output_processor = _OutputHandler( - window_fn=window_fn, - main_receivers=tagged_receivers[None], - tagged_receivers=tagged_receivers, - per_element_output_counter=None, - output_batch_converter=None, - process_yields_batches=False, - process_batch_yields_elements=False) - - do_fn_invoker = DoFnInvoker.create_invoker( - do_fn_signature, - output_processor, - context, - None, - args, - kwargs, + do_fn_invoker_args = [ + DoFnSignature(transform.fn), + _OutputHandler( + window_fn=window_fn, + main_receivers=tagged_receivers[None], + tagged_receivers=tagged_receivers, + per_element_output_counter=None, + output_batch_converter=None, + process_yields_batches=False, + process_batch_yields_elements=False), + ] + do_fn_invoker_kwargs = dict( + context=DoFnContext(transform.label, state=None), + side_inputs=None, + input_args=args, + input_kwargs=kwargs, user_state_context=None, - bundle_finalizer_param=bundle_finalizer_param) - - def apply_dofn_to_bundle(items): - do_fn_invoker.invoke_setup() - do_fn_invoker.invoke_start_bundle() - - for it in items: - do_fn_invoker.invoke_process(it) - - results = [v.value for v in tagged_receivers.values] - - do_fn_invoker.invoke_finish_bundle() - do_fn_invoker.invoke_teardown() - - return results - - return input_bag.map(get_windowed_value, - window_fn).map_partitions(apply_dofn_to_bundle) + bundle_finalizer_param=DoFn.BundleFinalizerParam(), + ) + + return input_bag.map(get_windowed_value, window_fn).map_partitions( + apply_dofn_to_bundle, + do_fn_invoker_args, + do_fn_invoker_kwargs, + tagged_receivers, + ) class GroupByKey(DaskBagOp): From bfbbf1c6596457eacb80b983a4c089b3170c761d Mon Sep 17 00:00:00 2001 From: Charles Stern <62192187+cisaacstern@users.noreply.github.com> Date: Thu, 26 Oct 2023 22:37:00 -0700 Subject: [PATCH 142/173] gather futures to prevent silent errors --- sdks/python/apache_beam/runners/dask/dask_runner.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index 49ba405d2838..902ae4765ecb 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -105,6 +105,10 @@ def wait_until_finish(self, duration=None) -> str: # Convert milliseconds to seconds duration /= 1000 distributed.wait(self.futures, timeout=duration) + # worker errors are not raised on client without gathering, + # so we need to gather to ensure there are no errors, see: + # https://distributed.dask.org/en/stable/resilience.html#user-code-failures + self.client.gather(self.futures) self._state = PipelineState.DONE except: # pylint: disable=broad-except self._state = PipelineState.FAILED From f287a46c0edf7f960cb085754655b3e2e5c2ab78 Mon Sep 17 00:00:00 2001 From: Charles Stern <62192187+cisaacstern@users.noreply.github.com> Date: Fri, 27 Oct 2023 14:01:20 -0700 Subject: [PATCH 143/173] gather and wait with as_completed to reduce memory footprint of results --- .../apache_beam/runners/dask/dask_runner.py | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index 902ae4765ecb..32ad25da97cb 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -104,11 +104,16 @@ def wait_until_finish(self, duration=None) -> str: if duration is not None: # Convert milliseconds to seconds duration /= 1000 - distributed.wait(self.futures, timeout=duration) - # worker errors are not raised on client without gathering, - # so we need to gather to ensure there are no errors, see: - # https://distributed.dask.org/en/stable/resilience.html#user-code-failures - self.client.gather(self.futures) + for _ in distributed.as_completed(self.futures, + timeout=duration, + with_results=True): + # without gathering results, worker errors are not raised on the client: + # https://distributed.dask.org/en/stable/resilience.html#user-code-failures + # so we want to gather results to raise errors client-side, but we do + # not actually need to use the results here, so we just pass. to gather, + # we use the iterative `as_completed(..., with_results=True)`, instead + # of aggregate `client.gather`, to minimize memory footprint of results. + pass self._state = PipelineState.DONE except: # pylint: disable=broad-except self._state = PipelineState.FAILED From 83fbfc57e0b92591f0f0192d3fbed4a7bc8488f5 Mon Sep 17 00:00:00 2001 From: Charles Stern <62192187+cisaacstern@users.noreply.github.com> Date: Mon, 18 Dec 2023 09:24:25 -0800 Subject: [PATCH 144/173] install distributed from github, temporarily --- sdks/python/setup.py | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/sdks/python/setup.py b/sdks/python/setup.py index d3fca2147239..110f87289f13 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -418,8 +418,13 @@ def get_portability_package_data(): ], 'dataframe': dataframe_dependency, 'dask': [ - 'dask >= 2022.6', - 'distributed >= 2022.6', + # FIXME(cisaacstern): The git+ link below is where https://github.com/dask/distributed/pull/8400 + # was merged into `distributed`. This PR is a fix for https://github.com/apache/beam/issues/29365. + # Installing from here to move forward with development. Before merge, this should be replaced with + # a lower bound release of `distributed`, once a release that includes the linked PR is available. + # 'dask >= 2023.XX', + # 'distributed >= 2023.XX', + 'distributed @ git+https://github.com/dask/distributed.git@8c3eb6f0bf47d124c887c543599d80ff09c3f5ed', ], 'yaml': [ 'docstring-parser>=0.15,<1.0', From a106deceb1d10bada377dcd0264c2efc856e5821 Mon Sep 17 00:00:00 2001 From: Charles Stern <62192187+cisaacstern@users.noreply.github.com> Date: Mon, 18 Dec 2023 10:43:31 -0800 Subject: [PATCH 145/173] client.wait_for_workers requires n_workers arg; use as_completed instead --- .../apache_beam/runners/dask/dask_runner.py | 20 ++++++++++++++----- 1 file changed, 15 insertions(+), 5 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index 109c4379b45d..4de68066125f 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -86,21 +86,31 @@ def _add_argparse_args(cls, parser: argparse.ArgumentParser) -> None: @dataclasses.dataclass class DaskRunnerResult(PipelineResult): - from dask import distributed + import dask.distributed as ddist - client: distributed.Client - futures: t.Sequence[distributed.Future] + client: ddist.Client + futures: t.Sequence[ddist.Future] def __post_init__(self): super().__init__(PipelineState.RUNNING) def wait_until_finish(self, duration=None) -> str: + import dask.distributed as ddist + try: if duration is not None: # Convert milliseconds to seconds duration /= 1000 - self.client.wait_for_workers(timeout=duration) - self.client.gather(self.futures, errors='raise') + for _ in ddist.as_completed(self.futures, + timeout=duration, + with_results=True): + # without gathering results, worker errors are not raised on the client: + # https://distributed.dask.org/en/stable/resilience.html#user-code-failures + # so we want to gather results to raise errors client-side, but we do + # not actually need to use the results here, so we just pass. to gather, + # we use the iterative `as_completed(..., with_results=True)`, instead + # of aggregate `client.gather`, to minimize memory footprint of results. + pass self._state = PipelineState.DONE except: # pylint: disable=broad-except self._state = PipelineState.FAILED From 9067a54baf3741de3d144d77bfc2ed943b4e8236 Mon Sep 17 00:00:00 2001 From: Charles Stern <62192187+cisaacstern@users.noreply.github.com> Date: Thu, 21 Dec 2023 15:11:13 -0800 Subject: [PATCH 146/173] add test replicating #29365 bug --- .../apache_beam/runners/dask/dask_runner_test.py | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index d8b3e17d8a56..fd4e1cebc6f7 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -73,6 +73,11 @@ def test_create(self): pcoll = p | beam.Create([1]) assert_that(pcoll, equal_to([1])) + def test_create_multiple(self): + with self.pipeline as p: + pcoll = p | beam.Create([1, 2, 3, 4]) + assert_that(pcoll, equal_to([1, 2, 3, 4])) + def test_create_and_map(self): def double(x): return x * 2 @@ -89,6 +94,14 @@ def double(x): pcoll = p | beam.Create([1]) | beam.Map(double) | beam.GroupByKey() assert_that(pcoll, equal_to([(2, [1])])) + def test_groupby_string_keys(self): + with self.pipeline as p: + pcoll = ( + p + | beam.Create([('a', 1), ('a', 2), ('b', 3), ('b', 4)]) + | beam.GroupByKey()) + assert_that(pcoll, equal_to([('a', [1, 2]), ('b', [3, 4])])) + if __name__ == '__main__': unittest.main() From d6c35eb1e85136bbd06110acec9dd59770e76f35 Mon Sep 17 00:00:00 2001 From: Charles Stern <62192187+cisaacstern@users.noreply.github.com> Date: Thu, 21 Dec 2023 15:11:52 -0800 Subject: [PATCH 147/173] for development, re-pin to dask pr branch --- sdks/python/setup.py | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/sdks/python/setup.py b/sdks/python/setup.py index 110f87289f13..f1618818359a 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -418,13 +418,16 @@ def get_portability_package_data(): ], 'dataframe': dataframe_dependency, 'dask': [ - # FIXME(cisaacstern): The git+ link below is where https://github.com/dask/distributed/pull/8400 - # was merged into `distributed`. This PR is a fix for https://github.com/apache/beam/issues/29365. + # FIXME(cisaacstern): The git+ link below is a fix for https://github.com/apache/beam/issues/29365. # Installing from here to move forward with development. Before merge, this should be replaced with - # a lower bound release of `distributed`, once a release that includes the linked PR is available. - # 'dask >= 2023.XX', - # 'distributed >= 2023.XX', - 'distributed @ git+https://github.com/dask/distributed.git@8c3eb6f0bf47d124c887c543599d80ff09c3f5ed', + # a lower bound release of `dask` that includes https://github.com/dask/dask/pull/10734. + # 'dask >= 2024.XX.X', + # 'distributed >= 2024.XX.X', + 'dask @ git+https://github.com/cisaacstern/dask.git@tokenize-bag-groupby-key', + # For development, 'distributed >= 2023.12.1' should work with the above dask PR, however it can't + # be installed as part of a single `pip` call, since distributed releases are pinned to specific + # dask releases. As a workaround, distributed can be installed first, and then `.[dask]` installed + # second, with the `--update` / `-U` flag to replace the dask release brought in by distributed. ], 'yaml': [ 'docstring-parser>=0.15,<1.0', From 3c0fbc68d2734945b5a560557251dd2c290fc325 Mon Sep 17 00:00:00 2001 From: Charles Stern <62192187+cisaacstern@users.noreply.github.com> Date: Fri, 22 Dec 2023 14:57:06 -0800 Subject: [PATCH 148/173] side inputs for side_input test --- .../apache_beam/runners/dask/dask_runner_test.py | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index 43b04dbccfb1..3e725e7cfe62 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -125,20 +125,28 @@ def double(x): | beam.GroupByKey()) assert_that(pcoll, equal_to([(2, [1, 1]), (4, [2, 2]), (6, [3])])) - def test_map_with_side_inputs(self): + def test_map_with_positional_side_input(self): def mult_by(x, y): return x * y with self.pipeline as p: - pcoll = p | beam.Create([1]) | beam.Map(mult_by, 3) + side = p | "side" >> beam.Create([3]) + pcoll = ( + p + | "main" >> beam.Create([1]) + | beam.Map(mult_by, beam.pvalue.AsSingleton(side))) assert_that(pcoll, equal_to([3])) - def test_map_with_named_side_inputs(self): + def test_map_with_keyword_side_input(self): def mult_by(x, y): return x * y with self.pipeline as p: - pcoll = p | beam.Create([1]) | beam.Map(mult_by, y=3) + side = p | "side" >> beam.Create([3]) + pcoll = ( + p + | "main" >> beam.Create([1]) + | beam.Map(mult_by, y=beam.pvalue.AsSingleton(side))) assert_that(pcoll, equal_to([3])) def test_groupby_with_fixed_windows(self): From 8dc89718568ab917ca658ff8db4affcc1de828c1 Mon Sep 17 00:00:00 2001 From: Charles Stern <62192187+cisaacstern@users.noreply.github.com> Date: Fri, 22 Dec 2023 15:46:47 -0800 Subject: [PATCH 149/173] SideInputMap WIP --- .../runners/dask/transform_evaluator.py | 18 +++++++++++++++++- 1 file changed, 17 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index 1d18ce2c36ec..ba433e5049e5 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -41,6 +41,7 @@ from apache_beam.runners.dask.overrides import _Create from apache_beam.runners.dask.overrides import _Flatten from apache_beam.runners.dask.overrides import _GroupByKeyOnly +from apache_beam.transforms.sideinputs import SideInputMap from apache_beam.transforms.window import GlobalWindow from apache_beam.transforms.window import TimestampedValue from apache_beam.transforms.window import WindowFn @@ -176,6 +177,21 @@ def apply(self, input_bag: db.Bag) -> db.Bag: window_fn = main_input.windowing.windowfn if hasattr( main_input, "windowing") else None + # FIXME(cisaacstern): Snippet on side inputs below copied from RayRunner: + # https://github.com/ray-project/ray_beam_runner/blob/ecc9dba99dc9cbe51c2bbcb1fd472b288a89d1ba/ray_beam_runner/translator.py#L515-L524 + # This does not work yet, as I am not sure how to access an interable of + # actual values for the side_inputs yet. Commented-out `_collection_map` + # and related `RayDatasetAccessor` below may provide some design insight. + side_inputs = [] + for side_input in self.applied.side_inputs: + # side_ds = self._collection_map.get(side_input.pvalue) + side_inputs.append( + SideInputMap( + type(side_input), + side_input._view_options(), + # RayDatasetAccessor(side_ds, side_input._window_mapping_fn), + )) + tagged_receivers = OneReceiver() do_fn_invoker_args = [ @@ -191,7 +207,7 @@ def apply(self, input_bag: db.Bag) -> db.Bag: ] do_fn_invoker_kwargs = dict( context=DoFnContext(transform.label, state=None), - side_inputs=None, + side_inputs=side_inputs, input_args=args, input_kwargs=kwargs, user_state_context=None, From 327ba0dcdcaa05e0f664f09787ec7e53c265bced Mon Sep 17 00:00:00 2001 From: Charles Stern <62192187+cisaacstern@users.noreply.github.com> Date: Wed, 3 Jan 2024 12:15:01 -0800 Subject: [PATCH 150/173] add DaskBagWindowedIterator, and side_inputs to apply signature --- .../runners/dask/transform_evaluator.py | 44 ++++++++++--------- 1 file changed, 23 insertions(+), 21 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index ba433e5049e5..ea2df52192db 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -49,6 +49,7 @@ # Inputs to DaskOps. OpInput = t.Union[db.Bag, t.Sequence[db.Bag], None] +OpSide = t.Optional[t.Sequence[SideInputMap]] # Value types for PCollections (possibly Windowed Values). PCollVal = t.Union[WindowedValue, t.Any] @@ -78,6 +79,21 @@ def defenestrate(x): return x +@dataclasses.dataclass +class DaskBagWindowedIterator: + """Iterator for `apache_beam.transforms.sideinputs.SideInputMap`""" + + bag: db.Bag + window_fn: WindowFn + + def __iter__(self): + # FIXME(cisaacstern): list() is likely inefficient, since it presumably + # materializes the full result before iterating over it. doing this for + # now as a proof-of-concept. can we can generate results incrementally? + for result in list(self.bag): + yield get_windowed_value(result, self.window_fn) + + @dataclasses.dataclass class TaggingReceiver(Receiver): """A Receiver that handles tagged `WindowValue`s.""" @@ -121,19 +137,19 @@ def transform(self): return self.applied.transform @abc.abstractmethod - def apply(self, input_bag: OpInput) -> db.Bag: + def apply(self, input_bag: OpInput, side_inputs: OpSide = None) -> db.Bag: pass class NoOp(DaskBagOp): """An identity on a dask bag: returns the input as-is.""" - def apply(self, input_bag: OpInput) -> db.Bag: + def apply(self, input_bag: OpInput, side_inputs: OpSide = None) -> db.Bag: return input_bag class Create(DaskBagOp): """The beginning of a Beam pipeline; the input must be `None`.""" - def apply(self, input_bag: OpInput) -> db.Bag: + def apply(self, input_bag: OpInput, side_inputs: OpSide = None) -> db.Bag: assert input_bag is None, 'Create expects no input!' original_transform = t.cast(_Create, self.transform) items = original_transform.values @@ -168,7 +184,7 @@ class ParDo(DaskBagOp): This consumes a sequence of items and returns a sequence of items. """ - def apply(self, input_bag: db.Bag) -> db.Bag: + def apply(self, input_bag: db.Bag, side_inputs: OpSide = None) -> db.Bag: transform = t.cast(apache_beam.ParDo, self.transform) args, kwargs = transform.raw_side_inputs @@ -177,21 +193,6 @@ def apply(self, input_bag: db.Bag) -> db.Bag: window_fn = main_input.windowing.windowfn if hasattr( main_input, "windowing") else None - # FIXME(cisaacstern): Snippet on side inputs below copied from RayRunner: - # https://github.com/ray-project/ray_beam_runner/blob/ecc9dba99dc9cbe51c2bbcb1fd472b288a89d1ba/ray_beam_runner/translator.py#L515-L524 - # This does not work yet, as I am not sure how to access an interable of - # actual values for the side_inputs yet. Commented-out `_collection_map` - # and related `RayDatasetAccessor` below may provide some design insight. - side_inputs = [] - for side_input in self.applied.side_inputs: - # side_ds = self._collection_map.get(side_input.pvalue) - side_inputs.append( - SideInputMap( - type(side_input), - side_input._view_options(), - # RayDatasetAccessor(side_ds, side_input._window_mapping_fn), - )) - tagged_receivers = OneReceiver() do_fn_invoker_args = [ @@ -224,7 +225,7 @@ def apply(self, input_bag: db.Bag) -> db.Bag: class GroupByKey(DaskBagOp): """Group a PCollection into a mapping of keys to elements.""" - def apply(self, input_bag: db.Bag) -> db.Bag: + def apply(self, input_bag: db.Bag, side_inputs: OpSide = None) -> db.Bag: def key(item): return item[0] @@ -237,7 +238,8 @@ def value(item): class Flatten(DaskBagOp): """Produces a flattened bag from a collection of bags.""" - def apply(self, input_bag: t.List[db.Bag]) -> db.Bag: + def apply( + self, input_bag: t.List[db.Bag], side_inputs: OpSide = None) -> db.Bag: assert isinstance(input_bag, list), 'Must take a sequence of bags!' return db.concat(input_bag) From 21573a344f82395799c41a4508495cf1ef67e0eb Mon Sep 17 00:00:00 2001 From: Charles Stern <62192187+cisaacstern@users.noreply.github.com> Date: Wed, 3 Jan 2024 12:20:12 -0800 Subject: [PATCH 151/173] in visitor, apply side_inputs --- .../apache_beam/runners/dask/dask_runner.py | 25 +++++++++++++++---- 1 file changed, 20 insertions(+), 5 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index b457e6273c52..a27bbfefa87c 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -31,10 +31,12 @@ from apache_beam.pipeline import PipelineVisitor from apache_beam.runners.dask.overrides import dask_overrides from apache_beam.runners.dask.transform_evaluator import TRANSLATIONS +from apache_beam.runners.dask.transform_evaluator import DaskBagWindowedIterator from apache_beam.runners.dask.transform_evaluator import NoOp from apache_beam.runners.direct.direct_runner import BundleBasedDirectRunner from apache_beam.runners.runner import PipelineResult from apache_beam.runners.runner import PipelineState +from apache_beam.transforms.sideinputs import SideInputMap from apache_beam.utils.interactive_utils import is_in_notebook try: @@ -147,6 +149,7 @@ def visit_transform(self, transform_node: AppliedPTransform) -> None: op_class = TRANSLATIONS.get(transform_node.transform.__class__, NoOp) op = op_class(transform_node) + op_kws = {"input_bag": None, "side_inputs": None} inputs = list(transform_node.inputs) if inputs: bag_inputs = [] @@ -159,12 +162,24 @@ def visit_transform(self, transform_node: AppliedPTransform) -> None: bag_inputs.append(self.bags[prev_op]) if len(bag_inputs) == 1: - self.bags[transform_node] = op.apply(bag_inputs[0]) + op_kws["input_bag"] = bag_inputs[0] else: - self.bags[transform_node] = op.apply(bag_inputs) - - else: - self.bags[transform_node] = op.apply(None) + op_kws["input_bag"] = bag_inputs + + side_inputs = list(transform_node.side_inputs) + if side_inputs: + bag_side_inputs = [] + for si in side_inputs: + si_asbag = self.bags.get(si.pvalue.producer) + bag_side_inputs.append( + SideInputMap( + type(si), + si._view_options(), + DaskBagWindowedIterator(si_asbag, si._window_mapping_fn))) + + op_kws["side_inputs"] = bag_side_inputs + + self.bags[transform_node] = op.apply(**op_kws) return DaskBagVisitor() From 9bf08a717afe66c88418a9947b42b6a8b95b9ab5 Mon Sep 17 00:00:00 2001 From: Charles Stern <62192187+cisaacstern@users.noreply.github.com> Date: Wed, 3 Jan 2024 15:55:42 -0800 Subject: [PATCH 152/173] add ray runner side inputs tests --- .../runners/dask/dask_runner_test.py | 209 ++++++++++++++++++ 1 file changed, 209 insertions(+) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index 3e725e7cfe62..cb4975b8b59d 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -17,6 +17,7 @@ import datetime import inspect import unittest +import typing as t import apache_beam as beam from apache_beam.options.pipeline_options import PipelineOptions @@ -149,6 +150,201 @@ def mult_by(x, y): | beam.Map(mult_by, y=beam.pvalue.AsSingleton(side))) assert_that(pcoll, equal_to([3])) + def test_pardo_side_inputs(self): + def cross_product(elem, sides): + for side in sides: + yield elem, side + + with self.pipeline as p: + main = p | "main" >> beam.Create(["a", "b", "c"]) + side = p | "side" >> beam.Create(["x", "y"]) + assert_that( + main | beam.FlatMap(cross_product, beam.pvalue.AsList(side)), + equal_to([ + ("a", "x"), + ("b", "x"), + ("c", "x"), + ("a", "y"), + ("b", "y"), + ("c", "y"), + ]), + ) + + def test_pardo_side_input_dependencies(self): + with self.pipeline as p: + inputs = [p | beam.Create([None])] + for k in range(1, 10): + inputs.append( + inputs[0] + | beam.ParDo( + ExpectingSideInputsFn(f"Do{k}"), + *[beam.pvalue.AsList(inputs[s]) for s in range(1, k)], + )) + + def test_pardo_side_input_sparse_dependencies(self): + with self.pipeline as p: + inputs = [] + + def choose_input(s): + return inputs[(389 + s * 5077) % len(inputs)] + + for k in range(20): + num_inputs = int((k * k % 16)**0.5) + if num_inputs == 0: + inputs.append(p | f"Create{k}" >> beam.Create([f"Create{k}"])) + else: + inputs.append( + choose_input(0) + | beam.ParDo( + ExpectingSideInputsFn(f"Do{k}"), + *[ + beam.pvalue.AsList(choose_input(s)) + for s in range(1, num_inputs) + ], + )) + + def test_pardo_windowed_side_inputs(self): + with self.pipeline as p: + # Now with some windowing. + pcoll = ( + p + | beam.Create(list(range(10))) + | beam.Map(lambda t: window.TimestampedValue(t, t))) + # Intentionally choosing non-aligned windows to highlight the transition. + main = pcoll | "WindowMain" >> beam.WindowInto(window.FixedWindows(5)) + side = pcoll | "WindowSide" >> beam.WindowInto(window.FixedWindows(7)) + res = main | beam.Map( + lambda x, s: (x, sorted(s)), beam.pvalue.AsList(side)) + assert_that( + res, + equal_to([ + # The window [0, 5) maps to the window [0, 7). + (0, list(range(7))), + (1, list(range(7))), + (2, list(range(7))), + (3, list(range(7))), + (4, list(range(7))), + # The window [5, 10) maps to the window [7, 14). + (5, list(range(7, 10))), + (6, list(range(7, 10))), + (7, list(range(7, 10))), + (8, list(range(7, 10))), + (9, list(range(7, 10))), + ]), + label="windowed", + ) + + def test_flattened_side_input(self, with_transcoding=True): + with self.pipeline as p: + main = p | "main" >> beam.Create([None]) + side1 = p | "side1" >> beam.Create([("a", 1)]) + side2 = p | "side2" >> beam.Create([("b", 2)]) + if with_transcoding: + # Also test non-matching coder types (transcoding required) + third_element = [("another_type")] + else: + third_element = [("b", 3)] + side3 = p | "side3" >> beam.Create(third_element) + side = (side1, side2) | beam.Flatten() + assert_that( + main | beam.Map(lambda a, b: (a, b), beam.pvalue.AsDict(side)), + equal_to([(None, { + "a": 1, "b": 2 + })]), + label="CheckFlattenAsSideInput", + ) + assert_that( + (side, side3) | "FlattenAfter" >> beam.Flatten(), + equal_to([("a", 1), ("b", 2)] + third_element), + label="CheckFlattenOfSideInput", + ) + + def test_gbk_side_input(self): + with self.pipeline as p: + main = p | "main" >> beam.Create([None]) + side = p | "side" >> beam.Create([("a", 1)]) | beam.GroupByKey() + assert_that( + main | beam.Map(lambda a, b: (a, b), beam.pvalue.AsDict(side)), + equal_to([(None, { + "a": [1] + })]), + ) + + def test_multimap_side_input(self): + with self.pipeline as p: + main = p | "main" >> beam.Create(["a", "b"]) + side = p | "side" >> beam.Create([("a", 1), ("b", 2), ("a", 3)]) + assert_that( + main + | beam.Map( + lambda k, d: (k, sorted(d[k])), beam.pvalue.AsMultiMap(side)), + equal_to([("a", [1, 3]), ("b", [2])]), + ) + + def test_multimap_multiside_input(self): + # A test where two transforms in the same stage consume the same PCollection + # twice as side input. + with self.pipeline as p: + main = p | "main" >> beam.Create(["a", "b"]) + side = p | "side" >> beam.Create([("a", 1), ("b", 2), ("a", 3)]) + assert_that( + main + | "first map" >> beam.Map( + lambda k, + d, + l: (k, sorted(d[k]), sorted([e[1] for e in l])), + beam.pvalue.AsMultiMap(side), + beam.pvalue.AsList(side), + ) + | "second map" >> beam.Map( + lambda k, + d, + l: (k[0], sorted(d[k[0]]), sorted([e[1] for e in l])), + beam.pvalue.AsMultiMap(side), + beam.pvalue.AsList(side), + ), + equal_to([("a", [1, 3], [1, 2, 3]), ("b", [2], [1, 2, 3])]), + ) + + def test_multimap_side_input_type_coercion(self): + with self.pipeline as p: + main = p | "main" >> beam.Create(["a", "b"]) + # The type of this side-input is forced to Any (overriding type + # inference). Without type coercion to Tuple[Any, Any], the usage of this + # side-input in AsMultiMap() below should fail. + side = p | "side" >> beam.Create([("a", 1), ("b", 2), + ("a", 3)]).with_output_types(t.Any) + assert_that( + main + | beam.Map( + lambda k, d: (k, sorted(d[k])), beam.pvalue.AsMultiMap(side)), + equal_to([("a", [1, 3]), ("b", [2])]), + ) + + def test_pardo_unfusable_side_inputs(self): + def cross_product(elem, sides): + for side in sides: + yield elem, side + + with self.pipeline as p: + pcoll = p | beam.Create(["a", "b"]) + assert_that( + pcoll | beam.FlatMap(cross_product, beam.pvalue.AsList(pcoll)), + equal_to([("a", "a"), ("a", "b"), ("b", "a"), ("b", "b")]), + ) + + with self.pipeline as p: + pcoll = p | beam.Create(["a", "b"]) + derived = ((pcoll, ) + | beam.Flatten() + | beam.Map(lambda x: (x, x)) + | beam.GroupByKey() + | "Unkey" >> beam.Map(lambda kv: kv[0])) + assert_that( + pcoll | beam.FlatMap(cross_product, beam.pvalue.AsList(derived)), + equal_to([("a", "a"), ("a", "b"), ("b", "a"), ("b", "b")]), + ) + def test_groupby_with_fixed_windows(self): def double(x): return x * 2, x @@ -177,5 +373,18 @@ def test_groupby_string_keys(self): assert_that(pcoll, equal_to([('a', [1, 2]), ('b', [3, 4])])) +class ExpectingSideInputsFn(beam.DoFn): + def __init__(self, name): + self._name = name + + def default_label(self): + return self._name + + def process(self, element, *side_inputs): + if not all(list(s) for s in side_inputs): + raise ValueError(f"Missing data in side input {side_inputs}") + yield self._name + + if __name__ == '__main__': unittest.main() From b8c1aafda818f516fb0991c70a2ca9f245dde736 Mon Sep 17 00:00:00 2001 From: Charles Stern <62192187+cisaacstern@users.noreply.github.com> Date: Wed, 3 Jan 2024 16:46:18 -0800 Subject: [PATCH 153/173] fix unfusable test, and single-element flattens --- sdks/python/apache_beam/runners/dask/dask_runner.py | 7 +++++-- .../apache_beam/runners/dask/dask_runner_test.py | 13 +++++++++---- sdks/python/apache_beam/runners/dask/overrides.py | 9 ++++++++- 3 files changed, 22 insertions(+), 7 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index a27bbfefa87c..0b2d5de1944e 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -30,7 +30,7 @@ from apache_beam.pipeline import AppliedPTransform from apache_beam.pipeline import PipelineVisitor from apache_beam.runners.dask.overrides import dask_overrides -from apache_beam.runners.dask.transform_evaluator import TRANSLATIONS +from apache_beam.runners.dask.transform_evaluator import TRANSLATIONS, Flatten from apache_beam.runners.dask.transform_evaluator import DaskBagWindowedIterator from apache_beam.runners.dask.transform_evaluator import NoOp from apache_beam.runners.direct.direct_runner import BundleBasedDirectRunner @@ -161,7 +161,10 @@ def visit_transform(self, transform_node: AppliedPTransform) -> None: if prev_op in self.bags: bag_inputs.append(self.bags[prev_op]) - if len(bag_inputs) == 1: + # Input to `Flatten` could be of length 1, e.g. a single-element + # tuple: `(pcoll, ) | beam.Flatten()`. If so, we still pass it as + # an iterable, because `Flatten.apply` always takes an iterable. + if len(bag_inputs) == 1 and not isinstance(op, Flatten): op_kws["input_bag"] = bag_inputs[0] else: op_kws["input_bag"] = bag_inputs diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index cb4975b8b59d..d5c92d897643 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -327,22 +327,27 @@ def cross_product(elem, sides): yield elem, side with self.pipeline as p: - pcoll = p | beam.Create(["a", "b"]) + pcoll = p | "Create1" >> beam.Create(["a", "b"]) assert_that( - pcoll | beam.FlatMap(cross_product, beam.pvalue.AsList(pcoll)), + pcoll | + "FlatMap1" >> beam.FlatMap(cross_product, beam.pvalue.AsList(pcoll)), equal_to([("a", "a"), ("a", "b"), ("b", "a"), ("b", "b")]), + label="assert_that1", ) with self.pipeline as p: - pcoll = p | beam.Create(["a", "b"]) + pcoll = p | "Create2" >> beam.Create(["a", "b"]) + derived = ((pcoll, ) | beam.Flatten() | beam.Map(lambda x: (x, x)) | beam.GroupByKey() | "Unkey" >> beam.Map(lambda kv: kv[0])) assert_that( - pcoll | beam.FlatMap(cross_product, beam.pvalue.AsList(derived)), + pcoll | "FlatMap2" >> beam.FlatMap( + cross_product, beam.pvalue.AsList(derived)), equal_to([("a", "a"), ("a", "b"), ("b", "a"), ("b", "b")]), + label="assert_that2", ) def test_groupby_with_fixed_windows(self): diff --git a/sdks/python/apache_beam/runners/dask/overrides.py b/sdks/python/apache_beam/runners/dask/overrides.py index 46cb14cad870..b952834f12d7 100644 --- a/sdks/python/apache_beam/runners/dask/overrides.py +++ b/sdks/python/apache_beam/runners/dask/overrides.py @@ -94,7 +94,14 @@ def expand(self, input_or_inputs): class _Flatten(beam.PTransform): def expand(self, input_or_inputs): - is_bounded = all(pcoll.is_bounded for pcoll in input_or_inputs) + if isinstance(input_or_inputs, beam.PCollection): + # NOTE(cisaacstern): I needed this to avoid + # `TypeError: 'PCollection' object is not iterable` + # being raised by `all(...)` call below for single-element flattens, i.e., + # `(pcoll, ) | beam.Flatten() | ...` + is_bounded = input_or_inputs.is_bounded + else: + is_bounded = all(pcoll.is_bounded for pcoll in input_or_inputs) return beam.pvalue.PCollection(self.pipeline, is_bounded=is_bounded) From 7932802600901ea2fddeec5c56c886dd737b5b5f Mon Sep 17 00:00:00 2001 From: Charles Stern <62192187+cisaacstern@users.noreply.github.com> Date: Thu, 4 Jan 2024 16:55:55 -0800 Subject: [PATCH 154/173] xfail windowed side inputs test --- sdks/python/apache_beam/runners/dask/dask_runner_test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index d5c92d897643..2da5a5f3ed36 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -203,6 +203,7 @@ def choose_input(s): ], )) + @unittest.expectedFailure def test_pardo_windowed_side_inputs(self): with self.pipeline as p: # Now with some windowing. From 07e42744fca0c50854bc732c58c382ebc0408f5e Mon Sep 17 00:00:00 2001 From: Charles Stern <62192187+cisaacstern@users.noreply.github.com> Date: Mon, 8 Jan 2024 13:39:21 -0800 Subject: [PATCH 155/173] get pylint to pass --- .../apache_beam/runners/dask/dask_runner.py | 3 ++- .../runners/dask/dask_runner_test.py | 2 +- sdks/python/setup.py | 21 ++++++++++++------- 3 files changed, 16 insertions(+), 10 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index 0b2d5de1944e..5f7dd83a94d3 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -30,8 +30,9 @@ from apache_beam.pipeline import AppliedPTransform from apache_beam.pipeline import PipelineVisitor from apache_beam.runners.dask.overrides import dask_overrides -from apache_beam.runners.dask.transform_evaluator import TRANSLATIONS, Flatten +from apache_beam.runners.dask.transform_evaluator import TRANSLATIONS from apache_beam.runners.dask.transform_evaluator import DaskBagWindowedIterator +from apache_beam.runners.dask.transform_evaluator import Flatten from apache_beam.runners.dask.transform_evaluator import NoOp from apache_beam.runners.direct.direct_runner import BundleBasedDirectRunner from apache_beam.runners.runner import PipelineResult diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index 2da5a5f3ed36..6fb957049ef2 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -16,8 +16,8 @@ # import datetime import inspect -import unittest import typing as t +import unittest import apache_beam as beam from apache_beam.options.pipeline_options import PipelineOptions diff --git a/sdks/python/setup.py b/sdks/python/setup.py index f1618818359a..4551c064e20e 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -418,16 +418,21 @@ def get_portability_package_data(): ], 'dataframe': dataframe_dependency, 'dask': [ - # FIXME(cisaacstern): The git+ link below is a fix for https://github.com/apache/beam/issues/29365. - # Installing from here to move forward with development. Before merge, this should be replaced with - # a lower bound release of `dask` that includes https://github.com/dask/dask/pull/10734. + # FIXME(cisaacstern): The git+ link below is a fix for + # https://github.com/apache/beam/issues/29365. Installing from + # here to move forward with development. Before merge, this + # should be replaced with a lower bound release of `dask` that + # includes https://github.com/dask/dask/pull/10734. # 'dask >= 2024.XX.X', # 'distributed >= 2024.XX.X', - 'dask @ git+https://github.com/cisaacstern/dask.git@tokenize-bag-groupby-key', - # For development, 'distributed >= 2023.12.1' should work with the above dask PR, however it can't - # be installed as part of a single `pip` call, since distributed releases are pinned to specific - # dask releases. As a workaround, distributed can be installed first, and then `.[dask]` installed - # second, with the `--update` / `-U` flag to replace the dask release brought in by distributed. + 'dask @ git+https://github.com/cisaacstern/dask.git@tokenize-bag-groupby-key', # pylint: disable=line-too-long + # For development, 'distributed >= 2023.12.1' should work with + # the above dask PR, however it can't be installed as part of + # a single `pip` call, since distributed releases are pinned to + # specific dask releases. As a workaround, distributed can be + # installed first, and then `.[dask]` installed second, with the + # `--update` / `-U` flag to replace the dask release brought in + # by distributed. ], 'yaml': [ 'docstring-parser>=0.15,<1.0', From 4067c9a7ef39a5d84910e232fa199239a28183fa Mon Sep 17 00:00:00 2001 From: Charles Stern <62192187+cisaacstern@users.noreply.github.com> Date: Wed, 10 Jan 2024 13:25:24 -0800 Subject: [PATCH 156/173] latest dask doesnt support py38, so mock it for autodoc --- sdks/python/scripts/generate_pydoc.sh | 2 +- sdks/python/tox.ini | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/python/scripts/generate_pydoc.sh b/sdks/python/scripts/generate_pydoc.sh index bff479f88e0a..20b2090a96a8 100755 --- a/sdks/python/scripts/generate_pydoc.sh +++ b/sdks/python/scripts/generate_pydoc.sh @@ -135,7 +135,7 @@ autodoc_member_order = 'bysource' autodoc_mock_imports = ["tensorrt", "cuda", "torch", "onnxruntime", "onnx", "tensorflow", "tensorflow_hub", "tensorflow_transform", "tensorflow_metadata", "transformers", "xgboost", "datatable", "transformers", - "sentence_transformers", + "sentence_transformers", "dask", ] # Allow a special section for documenting DataFrame API diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini index 579cc7332285..af757b647fa3 100644 --- a/sdks/python/tox.ini +++ b/sdks/python/tox.ini @@ -142,7 +142,7 @@ commands = [testenv:py38-docs] -extras = test,gcp,docs,interactive,dataframe,dask +extras = test,gcp,docs,interactive,dataframe deps = Sphinx==1.8.5 sphinx_rtd_theme==0.4.3 From 7d04b169b95533ec07d1cf049f6efc3d0dea20da Mon Sep 17 00:00:00 2001 From: Charles Stern <62192187+cisaacstern@users.noreply.github.com> Date: Wed, 10 Jan 2024 16:38:14 -0800 Subject: [PATCH 157/173] reenable tests, no py38, temporary commands_pre installs --- sdks/python/test-suites/tox/common.gradle | 5 ++--- sdks/python/tox.ini | 12 +++++++++--- 2 files changed, 11 insertions(+), 6 deletions(-) diff --git a/sdks/python/test-suites/tox/common.gradle b/sdks/python/test-suites/tox/common.gradle index 3fdd0c0c553b..0f1948ec265b 100644 --- a/sdks/python/test-suites/tox/common.gradle +++ b/sdks/python/test-suites/tox/common.gradle @@ -26,9 +26,8 @@ test.dependsOn "testPython${pythonVersionSuffix}" toxTask "testPy${pythonVersionSuffix}Cloud", "py${pythonVersionSuffix}-cloud", "${posargs}" test.dependsOn "testPy${pythonVersionSuffix}Cloud" -// toxTask "testPy${pythonVersionSuffix}Dask", "py${pythonVersionSuffix}-dask", "${posargs}" -// test.dependsOn "testPy${pythonVersionSuffix}Dask" - +toxTask "testPy${pythonVersionSuffix}Dask", "py${pythonVersionSuffix}-dask", "${posargs}" +test.dependsOn "testPy${pythonVersionSuffix}Dask" toxTask "testPy38CloudCoverage", "py38-cloudcoverage", "${posargs}" test.dependsOn "testPy38CloudCoverage" diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini index af757b647fa3..13a3452e9562 100644 --- a/sdks/python/tox.ini +++ b/sdks/python/tox.ini @@ -17,7 +17,7 @@ [tox] # new environments will be excluded by default unless explicitly added to envlist. -envlist = py38,py39,py310,py311,py38-{cloud,docs,lint,mypy,cloudcoverage,dask},py39-{cloud,dask},py310-{cloud,dask},py311-{cloud,dask},whitespacelint +envlist = py38,py39,py310,py311,py38-{cloud,docs,lint,mypy,cloudcoverage},py39-{cloud,dask},py310-{cloud,dask},py311-{cloud,dask},whitespacelint toxworkdir = {toxinidir}/target/{env:ENV_NAME:.tox} [pycodestyle] @@ -86,10 +86,16 @@ commands = python apache_beam/examples/complete/autocomplete_test.py bash {toxinidir}/scripts/run_pytest.sh {envname} "{posargs}" -[testenv:py{38,39,310,311}-dask] +[testenv:py{39,310,311}-dask] extras = test,dask,dataframes +commands_pre = + # FIXME(cisaacstern): remove this block before merge. Adding for now to get tests to pass in + # CI during development. See related note in setup.py; `distributed` needs to be installed + # before `dask` for development env to work correctly. + pip install 'distributed==2023.12.1' + pip install -U 'dask @ git+https://github.com/cisaacstern/dask.git@tokenize-bag-groupby-key' commands = - bash {toxinidir}/scripts/run_pytest.sh {envname} "{posargs}" + bash {toxinidir}/scripts/run_pytest.sh {envname} {toxinidir}/apache_beam/runners/dask/ [testenv:py38-cloudcoverage] deps = From 0ed78d5bbc3ed4fb7e7b7eaae6e9c1d2ea0dab56 Mon Sep 17 00:00:00 2001 From: Charles Stern <62192187+cisaacstern@users.noreply.github.com> Date: Wed, 10 Jan 2024 16:54:59 -0800 Subject: [PATCH 158/173] seems like pull_request_target is standard for other workflows --- .github/workflows/dask_runner_tests.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/dask_runner_tests.yml b/.github/workflows/dask_runner_tests.yml index c1bfdbd85488..b3e193563a60 100644 --- a/.github/workflows/dask_runner_tests.yml +++ b/.github/workflows/dask_runner_tests.yml @@ -22,7 +22,7 @@ name: Dask Runner Tests on: schedule: - cron: '3 7 * * *' - pull_request: + pull_request_target: branches: ['master', 'release-*'] tags: 'v*' paths: ['sdks/python/apache_beam/runners/dask/**'] From 810076eb72dc13b3eb478e0b37bde249edc10824 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Thu, 24 Oct 2024 21:19:54 +0900 Subject: [PATCH 159/173] Setting minimum dask version to release with @cisaacstern's fix. --- sdks/python/setup.py | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/sdks/python/setup.py b/sdks/python/setup.py index 8b9991868981..3b45cbf82fc1 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -512,14 +512,8 @@ def get_portability_package_data(): ], 'dataframe': dataframe_dependency, 'dask': [ - # FIXME(cisaacstern): The git+ link below is a fix for - # https://github.com/apache/beam/issues/29365. Installing from - # here to move forward with development. Before merge, this - # should be replaced with a lower bound release of `dask` that - # includes https://github.com/dask/dask/pull/10734. - # 'dask >= 2024.XX.X', - # 'distributed >= 2024.XX.X', - 'dask @ git+https://github.com/cisaacstern/dask.git@tokenize-bag-groupby-key', # pylint: disable=line-too-long + 'distributed >= 2024.4.2', + 'dask >= 2024.4.2', # For development, 'distributed >= 2023.12.1' should work with # the above dask PR, however it can't be installed as part of # a single `pip` call, since distributed releases are pinned to From b5ac28c32ad94a505be2e2a755324f3d0e921ad2 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Thu, 24 Oct 2024 21:42:30 +0900 Subject: [PATCH 160/173] Passing tests, given new `assert_that` rule. --- sdks/python/apache_beam/runners/dask/dask_runner_test.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner_test.py b/sdks/python/apache_beam/runners/dask/dask_runner_test.py index 6fb957049ef2..66dda4a984f4 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner_test.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner_test.py @@ -322,7 +322,7 @@ def test_multimap_side_input_type_coercion(self): equal_to([("a", [1, 3]), ("b", [2])]), ) - def test_pardo_unfusable_side_inputs(self): + def test_pardo_unfusable_side_inputs__one(self): def cross_product(elem, sides): for side in sides: yield elem, side @@ -336,6 +336,11 @@ def cross_product(elem, sides): label="assert_that1", ) + def test_pardo_unfusable_side_inputs__two(self): + def cross_product(elem, sides): + for side in sides: + yield elem, side + with self.pipeline as p: pcoll = p | "Create2" >> beam.Create(["a", "b"]) From da50982b58b1a2acec329a67734a4ca50b8099fa Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Tue, 29 Oct 2024 12:40:32 +0900 Subject: [PATCH 161/173] Addressing feedback: triggering on PRs. --- .github/workflows/dask_runner_tests.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/dask_runner_tests.yml b/.github/workflows/dask_runner_tests.yml index ceb081331f7f..9a28854f2de3 100644 --- a/.github/workflows/dask_runner_tests.yml +++ b/.github/workflows/dask_runner_tests.yml @@ -22,7 +22,7 @@ name: Dask Runner Tests on: schedule: - cron: '3 7 * * *' - pull_request_target: + pull_request: branches: ['master', 'release-*'] tags: 'v*' paths: ['sdks/python/apache_beam/runners/dask/**'] From 1c4b4e14beeaadbf00bb1e1892618879f17cbf45 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Thu, 7 Nov 2024 10:42:44 +0800 Subject: [PATCH 162/173] Removed `commands-pre` from tox. --- sdks/python/tox.ini | 6 ------ 1 file changed, 6 deletions(-) diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini index aea9f31a3246..958c6626936c 100644 --- a/sdks/python/tox.ini +++ b/sdks/python/tox.ini @@ -110,12 +110,6 @@ commands = [testenv:py{39,310,311,312}-dask] extras = test,dask,dataframes -commands_pre = - # FIXME(cisaacstern): remove this block before merge. Adding for now to get tests to pass in - # CI during development. See related note in setup.py; `distributed` needs to be installed - # before `dask` for development env to work correctly. - pip install 'distributed==2023.12.1' - pip install -U 'dask @ git+https://github.com/cisaacstern/dask.git@tokenize-bag-groupby-key' commands = bash {toxinidir}/scripts/run_pytest.sh {envname} {toxinidir}/apache_beam/runners/dask/ From 99ca8f2cab997407a9df3fced11efea65239fb5c Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Thu, 7 Nov 2024 10:44:42 +0800 Subject: [PATCH 163/173] Cover up the right import error. --- sdks/python/apache_beam/runners/dask/dask_runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dask/dask_runner.py b/sdks/python/apache_beam/runners/dask/dask_runner.py index 5f7dd83a94d3..0f2317074cea 100644 --- a/sdks/python/apache_beam/runners/dask/dask_runner.py +++ b/sdks/python/apache_beam/runners/dask/dask_runner.py @@ -45,7 +45,7 @@ # https://github.com/pytest-dev/pytest/issues/3216#issuecomment-1502451456 import dask.distributed as ddist except ImportError: - distributed = {} + ddist = {} class DaskOptions(PipelineOptions): From 596d3b95f2c89fe4bee4b2d016df7c23557483a4 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Thu, 7 Nov 2024 10:45:59 +0800 Subject: [PATCH 164/173] add back windows to dask runner test. --- .github/workflows/dask_runner_tests.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/dask_runner_tests.yml b/.github/workflows/dask_runner_tests.yml index 9a28854f2de3..cfae8342001c 100644 --- a/.github/workflows/dask_runner_tests.yml +++ b/.github/workflows/dask_runner_tests.yml @@ -62,7 +62,7 @@ jobs: strategy: fail-fast: false matrix: - os: [ubuntu-latest, macos-latest] + os: [ubuntu-latest, macos-latest, windows-latest] params: [ {"py_ver": "3.9", "tox_env": "py39"}, {"py_ver": "3.10", "tox_env": "py310" }, From 24adfce33d436f6a69ccf4e81b2f0393bfc857b0 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Thu, 7 Nov 2024 11:53:00 +0800 Subject: [PATCH 165/173] min diff in dask runner tests, fix tox. --- .github/workflows/dask_runner_tests.yml | 4 ++++ sdks/python/tox.ini | 2 ++ 2 files changed, 6 insertions(+) diff --git a/.github/workflows/dask_runner_tests.yml b/.github/workflows/dask_runner_tests.yml index cfae8342001c..0f60c22b6aab 100644 --- a/.github/workflows/dask_runner_tests.yml +++ b/.github/workflows/dask_runner_tests.yml @@ -83,6 +83,10 @@ jobs: if: startsWith(matrix.os, 'ubuntu') || startsWith(matrix.os, 'macos') working-directory: ./sdks/python run: tox -c tox.ini -e ${{ matrix.params.tox_env }}-dask + - name: Run tests basic windows + if: startsWith(matrix.os, 'windows') + working-directory: ./sdks/python + run: tox -c tox.ini -e ${{ matrix.params.tox_env }}-win-dask - name: Upload test logs uses: actions/upload-artifact@v4 if: always() diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini index 958c6626936c..a08a551943a4 100644 --- a/sdks/python/tox.ini +++ b/sdks/python/tox.ini @@ -110,6 +110,8 @@ commands = [testenv:py{39,310,311,312}-dask] extras = test,dask,dataframes +commands_pre = + pip install 'distributed>=2024.4.' 'dask>=2024.4.2' commands = bash {toxinidir}/scripts/run_pytest.sh {envname} {toxinidir}/apache_beam/runners/dask/ From 24762cc42711af6585f0bc8ef5fce8cb76e59ca3 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Thu, 7 Nov 2024 12:25:31 +0800 Subject: [PATCH 166/173] Update tox.ini --- sdks/python/tox.ini | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini index a08a551943a4..2f1b39ef9d1a 100644 --- a/sdks/python/tox.ini +++ b/sdks/python/tox.ini @@ -111,7 +111,7 @@ commands = [testenv:py{39,310,311,312}-dask] extras = test,dask,dataframes commands_pre = - pip install 'distributed>=2024.4.' 'dask>=2024.4.2' + pip install 'distributed>=2024.4.2' 'dask>=2024.4.2' commands = bash {toxinidir}/scripts/run_pytest.sh {envname} {toxinidir}/apache_beam/runners/dask/ From bb1f7a74a1ab615b52a9658eb84e5f4d3dd4cad2 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Thu, 7 Nov 2024 17:05:23 +0800 Subject: [PATCH 167/173] Adding windows support for dask tests in tox. --- sdks/python/tox.ini | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini index a08a551943a4..f73692718eed 100644 --- a/sdks/python/tox.ini +++ b/sdks/python/tox.ini @@ -115,6 +115,15 @@ commands_pre = commands = bash {toxinidir}/scripts/run_pytest.sh {envname} {toxinidir}/apache_beam/runners/dask/ +[testenv:py{39,310,311,312}-win-dask] +commands_pre = + pip install 'distributed>=2024.4.' 'dask>=2024.4.2' +commands = + python apache_beam/examples/complete/autocomplete_test.py + bash {toxinidir}/scripts/run_pytest.sh {envname} {toxinidir}/apache_beam/runners/dask/ +install_command = {envbindir}/python.exe {envbindir}/pip.exe install --retries 10 {opts} {packages} +list_dependencies_command = {envbindir}/python.exe {envbindir}/pip.exe freeze + [testenv:py39-cloudcoverage] deps = pytest-cov==3.0.0 From 28c26bb8c82f58fe5844ccf6bed215a6630306d7 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Thu, 7 Nov 2024 17:29:48 +0800 Subject: [PATCH 168/173] Fixed old typo from bad merge. --- sdks/python/tox.ini | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini index 553c100b69f7..725adb0b1940 100644 --- a/sdks/python/tox.ini +++ b/sdks/python/tox.ini @@ -117,7 +117,7 @@ commands = [testenv:py{39,310,311,312}-win-dask] commands_pre = - pip install 'distributed>=2024.4.' 'dask>=2024.4.2' + pip install 'distributed>=2024.4.2' 'dask>=2024.4.2' commands = python apache_beam/examples/complete/autocomplete_test.py bash {toxinidir}/scripts/run_pytest.sh {envname} {toxinidir}/apache_beam/runners/dask/ From 3058876614bb42cca04f48b8827630e0f476155f Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Fri, 8 Nov 2024 06:48:52 +0800 Subject: [PATCH 169/173] Addressed Charles's FIXME - using iter. --- sdks/python/apache_beam/runners/dask/transform_evaluator.py | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index ea2df52192db..ac557fb0c700 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -87,10 +87,7 @@ class DaskBagWindowedIterator: window_fn: WindowFn def __iter__(self): - # FIXME(cisaacstern): list() is likely inefficient, since it presumably - # materializes the full result before iterating over it. doing this for - # now as a proof-of-concept. can we can generate results incrementally? - for result in list(self.bag): + for result in iter(self.bag): yield get_windowed_value(result, self.window_fn) From 78da1a04c056937fbe3101cef47638427b67a395 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Fri, 8 Nov 2024 06:59:32 +0800 Subject: [PATCH 170/173] Add docstring to function. --- sdks/python/apache_beam/runners/dask/transform_evaluator.py | 1 + 1 file changed, 1 insertion(+) diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index ac557fb0c700..c2580ac7af49 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -158,6 +158,7 @@ def apply(self, input_bag: OpInput, side_inputs: OpSide = None) -> db.Bag: def apply_dofn_to_bundle( items, do_fn_invoker_args, do_fn_invoker_kwargs, tagged_receivers): + """Invokes a DoFn within a bundle, which is implemented as a Dask partition.""" do_fn_invoker = DoFnInvoker.create_invoker( *do_fn_invoker_args, **do_fn_invoker_kwargs) From be190d2cdd2f00fe58c744656bb09f1c5b839947 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Fri, 8 Nov 2024 06:59:58 +0800 Subject: [PATCH 171/173] Add dask back to docs tox environment. --- sdks/python/tox.ini | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini index 725adb0b1940..ad5d7ec5505e 100644 --- a/sdks/python/tox.ini +++ b/sdks/python/tox.ini @@ -172,7 +172,7 @@ commands = [testenv:docs] -extras = test,gcp,docs,interactive,dataframe +extras = test,gcp,docs,interactive,dataframe,dask deps = Sphinx==7.4.7 sphinx_rtd_theme==3.0.1 From 7caba8494efc1841a8ea4b72b2933e0bc0f55e54 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Fri, 8 Nov 2024 07:50:55 +0800 Subject: [PATCH 172/173] fix lint --- sdks/python/apache_beam/runners/dask/transform_evaluator.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index c2580ac7af49..ca0159e1b2dd 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -158,7 +158,7 @@ def apply(self, input_bag: OpInput, side_inputs: OpSide = None) -> db.Bag: def apply_dofn_to_bundle( items, do_fn_invoker_args, do_fn_invoker_kwargs, tagged_receivers): - """Invokes a DoFn within a bundle, which is implemented as a Dask partition.""" + """Invokes a DoFn within a bundle, implemented as a Dask partition.""" do_fn_invoker = DoFnInvoker.create_invoker( *do_fn_invoker_args, **do_fn_invoker_kwargs) From 3430bff84694a2d828bbba3caf6752fb02b28c48 Mon Sep 17 00:00:00 2001 From: Alex Merose Date: Fri, 8 Nov 2024 10:11:22 +0800 Subject: [PATCH 173/173] Reverted the TODO, it seems necessary. --- sdks/python/apache_beam/runners/dask/transform_evaluator.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dask/transform_evaluator.py b/sdks/python/apache_beam/runners/dask/transform_evaluator.py index ca0159e1b2dd..e3bd5fd87763 100644 --- a/sdks/python/apache_beam/runners/dask/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/dask/transform_evaluator.py @@ -87,7 +87,10 @@ class DaskBagWindowedIterator: window_fn: WindowFn def __iter__(self): - for result in iter(self.bag): + # FIXME(cisaacstern): list() is likely inefficient, since it presumably + # materializes the full result before iterating over it. doing this for + # now as a proof-of-concept. can we can generate results incrementally? + for result in list(self.bag): yield get_windowed_value(result, self.window_fn)