-
Notifications
You must be signed in to change notification settings - Fork 5.3k
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
Time backpressure #43110
Time backpressure #43110
Changes from 10 commits
9b69978
02aaf21
d29e00d
925c173
0fd0bb2
8ce7724
dc5dee4
296639d
2cc33a3
d3cacbe
ad3b6d0
d1cc9d5
8a836c7
42d65f4
4a861f5
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 |
---|---|---|
|
@@ -523,14 +523,19 @@ def select_operator_to_run( | |
ops = [] | ||
for op, state in topology.items(): | ||
under_resource_limits = _execution_allowed(op, resource_manager) | ||
in_backpressure = ( | ||
any(not p.can_add_input(op) for p in backpressure_policies) | ||
or not under_resource_limits | ||
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. nit, swap the order of these 2 conditions. so when |
||
) | ||
if ( | ||
under_resource_limits | ||
not in_backpressure | ||
and not op.completed() | ||
and state.num_queued() > 0 | ||
and op.should_add_input() | ||
and all(p.can_add_input(op) for p in backpressure_policies) | ||
): | ||
ops.append(op) | ||
# Signal whether op in backpressure for stats collections | ||
op.notify_in_task_submission_backpressure(in_backpressure) | ||
# Update the op in all cases to enable internal autoscaling, etc. | ||
op.notify_resource_usage(state.num_queued(), under_resource_limits) | ||
|
||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -120,7 +120,7 @@ def test_e2e_normal(self): | |
map_func1 = self._get_map_func(actor, 1) | ||
map_func2 = self._get_map_func(actor, 2) | ||
|
||
# Creat a dataset with 2 map ops. Each map op has N tasks, where N is | ||
# Create a dataset with 2 map ops. Each map op has N tasks, where N is | ||
# the number of cluster CPUs. | ||
N = self.__class__._cluster_cpus | ||
ds = ray.data.range(N, parallelism=N) | ||
|
@@ -138,6 +138,36 @@ def test_e2e_normal(self): | |
start2, end2 = ray.get(actor.get_start_and_end_time_for_op.remote(2)) | ||
assert start1 < start2 < end1 < end2, (start1, start2, end1, end2) | ||
|
||
def test_e2e_time_backpressure(self): | ||
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. maybe move this outside of 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. Going to leave as is to prevent having to duplicate some of the class methods outside the class. Once the metric is being exported, I will remove this duplicated test and include the assert from this test in each of the backpressure policy tests. |
||
"""A simple E2E test with ConcurrencyCapBackpressurePolicy enabled.""" | ||
# TODO: merge this test with the above once we are exporting the | ||
# backpressure_time in op_runtime_metrics.py. This test currently | ||
# mutates the OpRuntimeMetrics dataclass to observe the backpressure time. | ||
from ray.data._internal.execution.interfaces.op_runtime_metrics import ( | ||
OpRuntimeMetrics, | ||
) | ||
|
||
with patch.object( | ||
OpRuntimeMetrics.__dataclass_fields__["task_submission_backpressure_time"], | ||
"metadata", | ||
{"export": True}, | ||
): | ||
actor = self._create_record_time_actor() | ||
map_func1 = self._get_map_func(actor, 1) | ||
map_func2 = self._get_map_func(actor, 2) | ||
|
||
# Create a dataset with 2 map ops. Each map op has N tasks, where N is | ||
# the number of cluster CPUs. | ||
N = self.__class__._cluster_cpus | ||
ds = ray.data.range(N, parallelism=N) | ||
# Use different `num_cpus` to make sure they don't fuse. | ||
ds = ds.map_batches(map_func1, batch_size=None, num_cpus=1, concurrency=1) | ||
ds = ds.map_batches(map_func2, batch_size=None, num_cpus=1.1, concurrency=1) | ||
ds.take_all() | ||
assert ( | ||
0 < ds._plan.stats().extra_metrics["task_submission_backpressure_time"] | ||
) | ||
|
||
|
||
class TestStreamOutputBackpressurePolicy(unittest.TestCase): | ||
"""Tests for StreamOutputBackpressurePolicy.""" | ||
|
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.
nit, if this is an internal-only field, we can move it to
__init__
.