Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Run a significantly smaller Dask Graph. #33806

Merged
merged 3 commits into from
Jan 30, 2025
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
11 changes: 8 additions & 3 deletions sdks/python/apache_beam/runners/dask/dask_runner.py
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
scheduler.
"""
import argparse
import collections
import dataclasses
import typing as t

Expand Down Expand Up @@ -143,8 +144,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=collections.OrderedDict)

def visit_transform(self, transform_node: AppliedPTransform) -> None:
op_class = TRANSLATIONS.get(transform_node.transform.__class__, NoOp)
Expand Down Expand Up @@ -212,6 +213,10 @@ 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()))
# The dictionary in this visitor keeps a mapping of every Beam
# PTransform to the equivalent Bag operation. This is highly
# redundant. Thus, we can get away with computing just the last
# value, which should be connected to the full Bag Task Graph.
opt_graph = dask.optimize(list(dask_visitor.bags.values())[-1])
futures = client.compute(opt_graph)
return DaskRunnerResult(client, futures)
2 changes: 0 additions & 2 deletions sdks/python/apache_beam/runners/dask/transform_evaluator.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,6 @@

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
Expand Down
Loading