-
Notifications
You must be signed in to change notification settings - Fork 7.3k
[data] implement streaming output backpressure #40387
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
Changes from all commits
6e7de8c
7881f51
b192a78
c5a8453
90afcfe
18df710
b746753
c08d041
75b6c23
1fc504f
0ea025a
65d449d
7a7b423
87e719c
dff7416
b946ccb
2d142ed
4b5bd0c
0667bd4
3ce4a8a
3ac3564
0a8aeb3
009cb82
ebe8e93
05e42db
222dc0b
584a699
abf42f8
6891f41
9632e84
8fa34ec
259983c
af82f99
8e38323
21b6d19
84539ba
a35f380
ebdc82f
d0769fb
7b71cc8
5aaa08e
41951fb
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 |
|---|---|---|
| @@ -1,14 +1,14 @@ | ||
| import logging | ||
| from abc import ABC, abstractmethod | ||
| from typing import TYPE_CHECKING | ||
| from typing import TYPE_CHECKING, Dict | ||
|
|
||
| import ray | ||
|
|
||
| if TYPE_CHECKING: | ||
| from ray.data._internal.execution.interfaces.physical_operator import ( | ||
| PhysicalOperator, | ||
| ) | ||
| from ray.data._internal.execution.streaming_executor_state import Topology | ||
| from ray.data._internal.execution.streaming_executor_state import OpState, Topology | ||
|
|
||
| logger = logging.getLogger(__name__) | ||
|
|
||
|
|
@@ -36,14 +36,35 @@ class BackpressurePolicy(ABC): | |
| def __init__(self, topology: "Topology"): | ||
| ... | ||
|
|
||
| @abstractmethod | ||
| def can_run(self, op: "PhysicalOperator") -> bool: | ||
| """Called when StreamingExecutor selects an operator to run in | ||
| `streaming_executor_state.select_operator_to_run()`. | ||
| def calculate_max_blocks_to_read_per_op( | ||
| self, topology: "Topology" | ||
| ) -> Dict["OpState", int]: | ||
| """Determine how many blocks of data we can read from each operator. | ||
| The `DataOpTask`s of the operators will stop reading blocks when the limit is | ||
| reached. Then the execution of these tasks will be paused when the streaming | ||
| generator backpressure threshold is reached. | ||
| Used in `streaming_executor_state.py::process_completed_tasks()`. | ||
|
|
||
| Returns: A dict mapping from each operator's OpState to the desired number of | ||
| blocks to read. For operators that are not in the dict, all available blocks | ||
| will be read. | ||
|
|
||
| Note: Only one backpressure policy that implements this method can be enabled | ||
| at a time. | ||
| """ | ||
| return {} | ||
|
|
||
| def can_add_input(self, op: "PhysicalOperator") -> bool: | ||
| """Determine if we can add a new input to the operator. If returns False, the | ||
| operator will be backpressured and will not be able to run new tasks. | ||
| Used in `streaming_executor_state.py::select_operator_to_run()`. | ||
|
|
||
| Returns: True if the operator can run, False otherwise. | ||
| Returns: True if we can add a new input to the operator, False otherwise. | ||
|
|
||
| Note, if multiple backpressure policies are enabled, the operator will be | ||
| backpressured if any of the policies returns False. | ||
| """ | ||
| ... | ||
| return True | ||
|
|
||
|
|
||
| class ConcurrencyCapBackpressurePolicy(BackpressurePolicy): | ||
|
|
@@ -103,7 +124,7 @@ def __init__(self, topology: "Topology"): | |
| for op, _ in topology.items(): | ||
| self._concurrency_caps[op] = self._init_cap | ||
|
|
||
| def can_run(self, op: "PhysicalOperator") -> bool: | ||
| def can_add_input(self, op: "PhysicalOperator") -> bool: | ||
| metrics = op.metrics | ||
| while metrics.num_tasks_finished >= ( | ||
| self._concurrency_caps[op] * self._cap_multiply_threshold | ||
|
|
@@ -113,3 +134,82 @@ def can_run(self, op: "PhysicalOperator") -> bool: | |
| f"Concurrency cap for {op} increased to {self._concurrency_caps[op]}" | ||
| ) | ||
| return metrics.num_tasks_running < self._concurrency_caps[op] | ||
|
|
||
|
|
||
| class StreamingOutputBackpressurePolicy(BackpressurePolicy): | ||
| """A backpressure policy that throttles the streaming outputs of the `DataOpTask`s. | ||
|
|
||
| The are 2 levels of configs to control the behavior: | ||
| - At the Ray Core level, we use | ||
| `MAX_BLOCKS_IN_GENERATOR_BUFFER` to limit the number of blocks buffered in | ||
| the streaming generator of each OpDataTask. When it's reached, the task will | ||
| be blocked at `yield` until the caller reads another `ObjectRef. | ||
| - At the Ray Data level, we use | ||
| `MAX_BLOCKS_IN_GENERATOR_BUFFER` to limit the number of blocks buffered in the | ||
| output queue of each operator. When it's reached, we'll stop reading from the | ||
| streaming generators of the op's tasks, and thus trigger backpressure at the | ||
| Ray Core level. | ||
|
|
||
| Thus, total number of buffered blocks for each operator can be | ||
| `MAX_BLOCKS_IN_GENERATOR_BUFFER * num_running_tasks + | ||
| MAX_BLOCKS_IN_OP_OUTPUT_QUEUE`. | ||
| """ | ||
|
|
||
| # The max number of blocks that can be buffered at the streaming generator | ||
| # of each `DataOpTask`. | ||
| MAX_BLOCKS_IN_GENERATOR_BUFFER = 10 | ||
| MAX_BLOCKS_IN_GENERATOR_BUFFER_CONFIG_KEY = ( | ||
| "backpressure_policies.streaming_output.max_blocks_in_generator_buffer" | ||
| ) | ||
| # The max number of blocks that can be buffered at the operator output queue | ||
| # (`OpState.outqueue`). | ||
| MAX_BLOCKS_IN_OP_OUTPUT_QUEUE = 20 | ||
| MAX_BLOCKS_IN_OP_OUTPUT_QUEUE_CONFIG_KEY = ( | ||
| "backpressure_policies.streaming_output.max_blocks_in_op_output_queue" | ||
| ) | ||
|
|
||
| def __init__(self, topology: "Topology"): | ||
| data_context = ray.data.DataContext.get_current() | ||
| self._max_num_blocks_in_streaming_gen_buffer = data_context.get_config( | ||
| self.MAX_BLOCKS_IN_GENERATOR_BUFFER_CONFIG_KEY, | ||
| self.MAX_BLOCKS_IN_GENERATOR_BUFFER, | ||
| ) | ||
| assert self._max_num_blocks_in_streaming_gen_buffer > 0 | ||
| # The `_generator_backpressure_num_objects` parameter should be | ||
| # `2 * self._max_num_blocks_in_streaming_gen_buffer` because we yield | ||
| # 2 objects for each block: the block and the block metadata. | ||
| data_context._task_pool_data_task_remote_args[ | ||
| "_generator_backpressure_num_objects" | ||
| ] = (2 * self._max_num_blocks_in_streaming_gen_buffer) | ||
|
|
||
| self._max_num_blocks_in_op_output_queue = data_context.get_config( | ||
| self.MAX_BLOCKS_IN_OP_OUTPUT_QUEUE_CONFIG_KEY, | ||
| self.MAX_BLOCKS_IN_OP_OUTPUT_QUEUE, | ||
| ) | ||
| assert self._max_num_blocks_in_op_output_queue > 0 | ||
|
|
||
| def calculate_max_blocks_to_read_per_op( | ||
| self, topology: "Topology" | ||
| ) -> Dict["OpState", int]: | ||
| max_blocks_to_read_per_op: Dict["OpState", int] = {} | ||
| downstream_num_active_tasks = 0 | ||
| for op, state in list(topology.items())[::-1]: | ||
| max_blocks_to_read_per_op[state] = ( | ||
| self._max_num_blocks_in_op_output_queue - state.outqueue_num_blocks() | ||
| ) | ||
| if downstream_num_active_tasks == 0: | ||
| # If all downstream operators are idle, it could be because no resources | ||
| # are available. In this case, we'll make sure to read at least one | ||
| # block to avoid deadlock. | ||
|
Comment on lines
+201
to
+203
Contributor
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. I am still unsure about this condition. "it could be" -> meaning that there cases where the downstream operators are idle but it is not a deadlock? Can we guarantee that it is a deadlock?
Contributor
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. Actually isn't exponential rampup one of the cases where it's not actually a deadlock? I guess it is okay to merge for now, but I think we should revisit this condition after 2.8.
Contributor
Author
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. yes, this is not 100% percent accurate. so the current strategy is conservative, it only unblocks one block at a time. I'll add a TODO here.
Contributor
Author
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. After a second thought, it seems that the only false positive case is when the executor just starts, the first op hasn't produced blocks for the second op to consume. |
||
| # TODO(hchen): `downstream_num_active_tasks == 0` doesn't necessarily | ||
| # mean no enough resources. One false positive case is when the upstream | ||
| # op hasn't produced any blocks for the downstream op to consume. | ||
| # In this case, at least reading one block is fine. | ||
| # If there are other false positive cases, we may want to make this | ||
| # deadlock check more accurate by directly checking resources. | ||
| max_blocks_to_read_per_op[state] = max( | ||
| max_blocks_to_read_per_op[state], | ||
| 1, | ||
| ) | ||
| downstream_num_active_tasks += len(op.get_active_tasks()) | ||
| return max_blocks_to_read_per_op | ||
Uh oh!
There was an error while loading. Please reload this page.