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

Fix DaskRunner GBK by bumping dask lower bound #29802

Closed
wants to merge 4 commits into from
Closed
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
20 changes: 15 additions & 5 deletions sdks/python/apache_beam/runners/dask/dask_runner.py
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
13 changes: 13 additions & 0 deletions sdks/python/apache_beam/runners/dask/dask_runner_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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()
12 changes: 10 additions & 2 deletions sdks/python/setup.py
Original file line number Diff line number Diff line change
Expand Up @@ -418,8 +418,16 @@ def get_portability_package_data():
],
'dataframe': dataframe_dependency,
'dask': [
'dask >= 2022.6',
'distributed >= 2022.6',
# 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.
],
'yaml': [
'docstring-parser>=0.15,<1.0',
Expand Down
Loading