-
Notifications
You must be signed in to change notification settings - Fork 5.8k
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
Deflake occasional deadlock in test_dataset.py::test_basic_actors[True] #21970
Changes from all commits
a564aac
dd27c02
25825ff
6772faf
ce0f8c5
938c121
885d9b1
6d45b17
654a226
9d4c39c
838bbbe
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -11,14 +11,20 @@ | |
from ray.data.dataset_pipeline import DatasetPipeline | ||
|
||
|
||
# Temporarily use an actor here to avoid ownership issues with tasks: | ||
# https://github.com/ray-project/ray/issues/20554 | ||
@ray.remote(num_cpus=0, placement_group=None) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Note that @stephanie-wang's PR that ports stage task launching to a threadpool will also fix this by launching all tasks from the driver, and she'll have to resolve some merge conflicts here. #21845 There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This might be an alternative to the thread pools as well, though it's not tolerant of the actor failure. |
||
def pipeline_stage(fn: Callable[[], Dataset[T]], context: DatasetContext) -> Dataset[T]: | ||
DatasetContext._set_current(context) | ||
try: | ||
prev = set_progress_bars(False) | ||
return fn() | ||
finally: | ||
set_progress_bars(prev) | ||
class _StageRunner: | ||
def run(self, fn: Callable[[], Dataset[T]], context: DatasetContext) -> Dataset[T]: | ||
DatasetContext._set_current(context) | ||
try: | ||
prev = set_progress_bars(False) | ||
# Force eager evaluation of all blocks in the pipeline stage. This | ||
# prevents resource deadlocks due to overlapping stage execution | ||
# (e.g., task -> actor stage). | ||
return fn().force_reads() | ||
finally: | ||
set_progress_bars(prev) | ||
|
||
|
||
class PipelineExecutor: | ||
|
@@ -27,8 +33,9 @@ def __init__(self, pipeline: "DatasetPipeline[T]"): | |
self._stages: List[ObjectRef[Dataset[Any]]] = [None] * ( | ||
len(self._pipeline._stages) + 1 | ||
) | ||
self._stage_runners = [_StageRunner.remote() for _ in self._stages] | ||
self._iter = iter(self._pipeline._base_iterable) | ||
self._stages[0] = pipeline_stage.remote( | ||
self._stages[0] = self._stage_runners[0].run.remote( | ||
next(self._iter), DatasetContext.get_current() | ||
) | ||
|
||
|
@@ -80,14 +87,14 @@ def __next__(self): | |
output = result | ||
else: | ||
fn = self._pipeline._stages[i] | ||
self._stages[i + 1] = pipeline_stage.remote( | ||
self._stages[i + 1] = self._stage_runners[i].run.remote( | ||
lambda: fn(result), DatasetContext.get_current() | ||
) | ||
|
||
# Pull a new element for the initial slot if possible. | ||
if self._stages[0] is None: | ||
try: | ||
self._stages[0] = pipeline_stage.remote( | ||
self._stages[0] = self._stage_runners[0].run.remote( | ||
next(self._iter), DatasetContext.get_current() | ||
) | ||
except StopIteration: | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,25 @@ | ||
import pytest | ||
|
||
import ray | ||
from ray.tests.conftest import * # noqa | ||
|
||
NUM_REPEATS = 10 | ||
NUM_TASKS = 10 | ||
|
||
|
||
# This test can be flaky if there is resource deadlock between the pipeline | ||
# stages. Run it a lot to ensure no regressions. | ||
def test_basic_actors(shutdown_only): | ||
ray.init(num_cpus=2) | ||
for _ in range(NUM_REPEATS): | ||
ds = ray.data.range(NUM_TASKS) | ||
ds = ds.window(blocks_per_window=1) | ||
assert sorted(ds.map(lambda x: x + 1, compute="actors").take()) == list( | ||
range(1, NUM_TASKS + 1) | ||
) | ||
|
||
|
||
if __name__ == "__main__": | ||
import sys | ||
|
||
sys.exit(pytest.main(["-v", __file__])) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nice!