-
Notifications
You must be signed in to change notification settings - Fork 5.5k
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
[Data] Cap op concurrency with exponential ramp-up #40275
Changes from 11 commits
2d580a7
ed288f3
78d2b10
b699382
c08f1b6
647128a
a517c30
37913b0
fc14b94
600f197
061faf8
1a35d2d
527606d
681b589
ec46dd1
26e59b4
04e4e5a
9443aeb
ee0dd39
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 |
---|---|---|
@@ -0,0 +1,102 @@ | ||
import logging | ||
import os | ||
from abc import ABC, abstractmethod | ||
from typing import TYPE_CHECKING | ||
|
||
if TYPE_CHECKING: | ||
from ray.data._internal.execution.interfaces.physical_operator import ( | ||
PhysicalOperator, | ||
) | ||
from ray.data._internal.execution.streaming_executor_state import Topology | ||
|
||
logger = logging.getLogger(__name__) | ||
|
||
|
||
# TODO(hchen): Enable ConcurrencyCapBackpressurePolicy by default. | ||
DEFAULT_BACKPRESSURE_POLICIES = [] | ||
|
||
|
||
def get_backpressure_policies(topology: "Topology"): | ||
return [policy(topology) for policy in DEFAULT_BACKPRESSURE_POLICIES] | ||
|
||
|
||
class BackpressurePolicy(ABC): | ||
"""Interface for back pressure policies.""" | ||
|
||
@abstractmethod | ||
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()`. | ||
|
||
Returns: True if the operator can run, False otherwise. | ||
""" | ||
... | ||
|
||
|
||
class ConcurrencyCapBackpressurePolicy(BackpressurePolicy): | ||
"""A backpressure policy that caps the concurrency of each operator. | ||
|
||
The concurrency cap limits the number of concurrently running tasks. | ||
It will be set to an intial value, and will ramp up exponentially. | ||
|
||
The concrete stategy is as follows: | ||
- Each PhysicalOperator is assigned an initial concurrency cap. | ||
- An PhysicalOperator can run new tasks if the number of running tasks is less than the cap. | ||
- When the number of finished tasks reaches a threshold, the concurrency cap will | ||
increase. | ||
""" | ||
|
||
# Environment variable to configure this policy. | ||
# The format is: "<init_cap>,<cap_multiply_threshold>,<cap_multiplier>" | ||
CONFIG_ENV_VAR = "RAY_DATA_CONCURRENCY_CAP_CONFIG" | ||
|
||
# The intial concurrency cap for each operator. | ||
INIT_CAP = 4 | ||
# When the number of finished tasks reaches this threshold, the concurrency cap | ||
# will be multiplied by the multiplier. | ||
CAP_MULTIPLY_THRESHOLD = 0.5 | ||
# The multiplier to multiply the concurrency cap by. | ||
CAP_MULTIPLIER = 2.0 | ||
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. another way is to define these constant on the file level, so we don't need to parse environment variables. It's also maybe easier for advanced users to test out. 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. The issue with constants is that if the executor doesn't run on the driver (e.g., on the SplitCoordinator actor), it's hard to change the configs. I've seen this issue for other configs that depend on constants. Do you know a good solution to bypass this issue? 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 think you'll need to put them into the DataContext. 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. @stephanie-wang @c21 If we put them into DataContext, I'd like to to save them in a dict and add a key-value interface in DataContext. The API will be something like: data_context.set("concucrrency_cap_backpressure_policy.init_cap", 4) 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. That sounds good to me for now while it's still experimental. Actually, could you prepend the name with "experimental" or something like that? Makes deprecation a bit smoother. |
||
|
||
def __init__(self, topology: "Topology"): | ||
self._concurrency_caps: dict["PhysicalOperator", float] = {} | ||
|
||
self._init_cap = self.INIT_CAP | ||
self._cap_multiplier = self.CAP_MULTIPLIER | ||
self._cap_multiply_threshold = self.CAP_MULTIPLY_THRESHOLD | ||
|
||
env_config = os.environ.get(self.CONFIG_ENV_VAR, "") | ||
if env_config: | ||
try: | ||
configs = env_config.split(",") | ||
self._init_cap = int(configs[0]) | ||
self._cap_multiply_threshold = float(configs[1]) | ||
self._cap_multiplier = float(configs[2]) | ||
assert self._init_cap > 0 | ||
assert 0 < self._cap_multiply_threshold <= 1 | ||
assert self._cap_multiplier > 1 | ||
except Exception as e: | ||
raise ValueError("Invalid concurrency cap config", env_config) from e | ||
|
||
logger.debug( | ||
"Concurrency cap config: " | ||
f"{self._init_cap}, {self._cap_multiply_threshold}, {self._cap_multiplier}" | ||
) | ||
|
||
for op, _ in topology.items(): | ||
self._concurrency_caps[op] = self._init_cap | ||
|
||
def can_run(self, op: "PhysicalOperator") -> bool: | ||
metrics = op.metrics | ||
while metrics.num_tasks_finished >= ( | ||
self._concurrency_caps[op] * self._cap_multiply_threshold | ||
): | ||
self._concurrency_caps[op] *= self._cap_multiplier | ||
logger.debug( | ||
f"Concurrency cap for {op} increased to {self._concurrency_caps[op]}" | ||
) | ||
return metrics.num_tasks_running < self._concurrency_caps[op] |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,92 @@ | ||
import unittest | ||
from contextlib import contextmanager | ||
from unittest.mock import MagicMock, patch | ||
|
||
from pyarrow.hdfs import os | ||
|
||
from ray.data._internal.execution.backpressure_policy import ( | ||
ConcurrencyCapBackpressurePolicy, | ||
) | ||
from ray.data._internal.execution.streaming_executor_state import Topology | ||
|
||
|
||
class TestConcurrencyCapBackpressurePolicy(unittest.TestCase): | ||
@contextmanager | ||
def _patch_env_var(self, value): | ||
with patch.dict( | ||
os.environ, {ConcurrencyCapBackpressurePolicy.CONFIG_ENV_VAR: value} | ||
): | ||
yield | ||
|
||
def test_basic(self): | ||
op = MagicMock() | ||
op.metrics = MagicMock( | ||
num_tasks_running=0, | ||
num_tasks_finished=0, | ||
) | ||
topology = {op: MagicMock()} | ||
|
||
init_cap = 4 | ||
cap_multiply_threshold = 0.5 | ||
cap_multiplier = 2.0 | ||
|
||
with self._patch_env_var( | ||
f"{init_cap},{cap_multiply_threshold},{cap_multiplier}" | ||
): | ||
policy = ConcurrencyCapBackpressurePolicy(topology) | ||
|
||
self.assertEqual(policy._concurrency_caps[op], 4) | ||
# Gradually increase num_tasks_running to the cap. | ||
for i in range(1, init_cap + 1): | ||
self.assertTrue(policy.can_run(op)) | ||
op.metrics.num_tasks_running = i | ||
# Now num_tasks_running reaches the cap, so can_run should return False. | ||
self.assertFalse(policy.can_run(op)) | ||
|
||
# If we increase num_task_finished to the threshold (4 * 0.5 = 2), | ||
# it should trigger the cap to increase. | ||
op.metrics.num_tasks_finished = init_cap * cap_multiply_threshold | ||
self.assertEqual(policy.can_run(op), True) | ||
self.assertEqual(policy._concurrency_caps[op], init_cap * cap_multiplier) | ||
|
||
# Now the cap is 8 (4 * 2). | ||
# If we increase num_tasks_finished directly to the next-level's threshold | ||
# (8 * 2 * 0.5 = 8), it should trigger the cap to increase twice. | ||
op.metrics.num_tasks_finished = ( | ||
policy._concurrency_caps[op] * cap_multiplier * cap_multiply_threshold | ||
) | ||
op.metrics.num_tasks_running = 0 | ||
self.assertEqual(policy.can_run(op), True) | ||
self.assertEqual(policy._concurrency_caps[op], init_cap * cap_multiplier ** 3) | ||
|
||
|
||
def test_env_var_config(self): | ||
topology = MagicMock(Topology) | ||
# Test good config. | ||
with self._patch_env_var("10,0.3,1.5"): | ||
policy = ConcurrencyCapBackpressurePolicy(topology) | ||
self.assertEqual(policy._init_cap, 10) | ||
self.assertEqual(policy._cap_multiply_threshold, 0.3) | ||
self.assertEqual(policy._cap_multiplier, 1.5) | ||
|
||
# Test bad configs. | ||
with self._patch_env_var("10,0.3"): | ||
with self.assertRaises(ValueError): | ||
policy = ConcurrencyCapBackpressurePolicy(topology) | ||
with self._patch_env_var("-1,0.3,1.5"): | ||
with self.assertRaises(ValueError): | ||
policy = ConcurrencyCapBackpressurePolicy(topology) | ||
with self._patch_env_var("10,1.1,1.5"): | ||
with self.assertRaises(ValueError): | ||
policy = ConcurrencyCapBackpressurePolicy(topology) | ||
with self._patch_env_var("10,0.3,0.5"): | ||
with self.assertRaises(ValueError): | ||
policy = ConcurrencyCapBackpressurePolicy(topology) | ||
|
||
|
||
if __name__ == "__main__": | ||
import sys | ||
|
||
import pytest | ||
|
||
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.
this is not blocking comment, but better to have separate environment variables.
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.
This is to facilitate internal tests in 2.9. When we officially release this feature, we will probably simplify the configs. So I'll keep it for now.