Skip to content

Commit 0de2118

Browse files
authored
[Data] Concurrency cap backpressure with tuning (Disabled) (#59519)
EWMA_ALPHA Update EWMA_ALPHA from 0.2->0.1. This makes adjusting level to be more in-favor of limiting concurrency by being more sensitive to downstreaming queuing. K_DEV Update K_DEV from 2.0->1.0. This makes stddev to be more in-favor of limiting concurrency by being more sensitive to downstreaming queuing. cherry-pick of #59392
1 parent 1736619 commit 0de2118

File tree

4 files changed

+127
-36
lines changed

4 files changed

+127
-36
lines changed

python/ray/data/_internal/execution/backpressure_policy/concurrency_cap_backpressure_policy.py

Lines changed: 16 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -41,18 +41,18 @@ class ConcurrencyCapBackpressurePolicy(BackpressurePolicy):
4141
"""
4242

4343
# Smoothing factor for the asymmetric EWMA (slow fall, faster rise).
44-
EWMA_ALPHA = env_float("RAY_DATA_CONCURRENCY_CAP_EWMA_ALPHA", 0.2)
44+
EWMA_ALPHA = env_float("RAY_DATA_CONCURRENCY_CAP_EWMA_ALPHA", 0.1)
4545
EWMA_ALPHA_UP = 1.0 - (1.0 - EWMA_ALPHA) ** 2 # fast rise
4646
# Deadband width in units of the EWMA absolute deviation estimate.
47-
K_DEV = env_float("RAY_DATA_CONCURRENCY_CAP_K_DEV", 2.0)
47+
K_DEV = env_float("RAY_DATA_CONCURRENCY_CAP_K_DEV", 1.0)
4848
# Factor to back off when the queue is too large.
4949
BACKOFF_FACTOR = env_float("RAY_DATA_CONCURRENCY_CAP_BACKOFF_FACTOR", 1)
5050
# Factor to ramp up when the queue is too small.
5151
RAMPUP_FACTOR = env_float("RAY_DATA_CONCURRENCY_CAP_RAMPUP_FACTOR", 1)
5252
# Threshold for per-Op object store budget (available) vs total
5353
# (available / total) ratio to enable dynamic output queue size backpressure.
54-
OBJECT_STORE_BUDGET_RATIO = env_float(
55-
"RAY_DATA_CONCURRENCY_CAP_OBJECT_STORE_BUDGET_RATIO", 0.1
54+
AVAILABLE_OBJECT_STORE_BUDGET_THRESHOLD = env_float(
55+
"RAY_DATA_CONCURRENCY_CAP_AVAILABLE_OBJECT_STORE_BUDGET_THRESHOLD", 0.1
5656
)
5757

5858
def __init__(self, *args, **kwargs):
@@ -93,7 +93,7 @@ def __init__(self, *args, **kwargs):
9393
dynamic_output_queue_size_backpressure_configs = (
9494
f", EWMA_ALPHA={self.EWMA_ALPHA}, K_DEV={self.K_DEV}, "
9595
f"BACKOFF_FACTOR={self.BACKOFF_FACTOR}, RAMPUP_FACTOR={self.RAMPUP_FACTOR}, "
96-
f"OBJECT_STORE_BUDGET_RATIO={self.OBJECT_STORE_BUDGET_RATIO}"
96+
f"AVAILABLE_OBJECT_STORE_BUDGET_THRESHOLD={self.AVAILABLE_OBJECT_STORE_BUDGET_THRESHOLD}"
9797
)
9898
logger.debug(
9999
f"ConcurrencyCapBackpressurePolicy caps: {self._concurrency_caps}, "
@@ -141,10 +141,16 @@ def can_add_input(self, op: "PhysicalOperator") -> bool:
141141
"""Return whether `op` may accept another input now."""
142142
num_tasks_running = op.metrics.num_tasks_running
143143

144-
# If not a MapOperator or feature disabled, just enforce configured cap.
144+
# Skip dynamic backpressure if:
145+
# - Not a MapOperator
146+
# - Not eligible for Op for Backpressure
147+
# - Dynamic backpressure based on output queue size is disabled
148+
# - Downstream is a materializing op which requires full materialization
145149
if (
146150
not isinstance(op, MapOperator)
151+
or not self._resource_manager.is_op_eligible(op)
147152
or not self.enable_dynamic_output_queue_size_backpressure
153+
or self._resource_manager.has_materializing_downstream_op(op)
148154
):
149155
return num_tasks_running < self._concurrency_caps[op]
150156

@@ -156,20 +162,17 @@ def can_add_input(self, op: "PhysicalOperator") -> bool:
156162
total_mem = op_usage.object_store_memory + op_budget.object_store_memory
157163
if total_mem == 0 or (
158164
op_budget.object_store_memory / total_mem
159-
> self.OBJECT_STORE_BUDGET_RATIO
165+
> self.AVAILABLE_OBJECT_STORE_BUDGET_THRESHOLD
160166
):
161167
# If the objectstore budget (available) to total
162168
# ratio is above threshold (10%), skip dynamic output queue size
163169
# backpressure, but still enforce the configured cap.
164170
return num_tasks_running < self._concurrency_caps[op]
165171

166172
# Current total queued bytes (this op + downstream)
167-
current_queue_size_bytes = (
168-
self._resource_manager.get_op_internal_object_store_usage(op)
169-
+ self._resource_manager.get_op_outputs_object_store_usage_with_downstream(
170-
op
171-
)
172-
)
173+
current_queue_size_bytes = self._resource_manager.get_mem_op_internal(
174+
op
175+
) + self._resource_manager.get_op_outputs_object_store_usage_with_downstream(op)
173176

174177
# Update EWMA state (level & dev) and compute effective cap. Note that
175178
# we don't update the EWMA state if the objectstore budget (available) vs total

python/ray/data/_internal/execution/resource_manager.py

Lines changed: 19 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -135,8 +135,8 @@ def _warn_about_object_store_memory_if_needed(self):
135135
):
136136
logger.warning(
137137
f"{WARN_PREFIX} Ray's object store is configured to use only "
138-
f"{object_store_fraction:.1%} of available memory ({object_store_memory/GiB:.1f}GiB "
139-
f"out of {total_memory/GiB:.1f}GiB total). For optimal Ray Data performance, "
138+
f"{object_store_fraction:.1%} of available memory ({object_store_memory / GiB:.1f}GiB "
139+
f"out of {total_memory / GiB:.1f}GiB total). For optimal Ray Data performance, "
140140
f"we recommend setting the object store to at least 50% of available memory. "
141141
f"You can do this by setting the 'object_store_memory' parameter when calling "
142142
f"ray.init() or by setting the RAY_DEFAULT_OBJECT_STORE_MEMORY_PROPORTION environment variable."
@@ -268,6 +268,14 @@ def get_op_usage(self, op: PhysicalOperator) -> ExecutionResources:
268268
"""Return the resource usage of the given operator at the current time."""
269269
return self._op_usages[op]
270270

271+
def get_mem_op_internal(self, op: PhysicalOperator) -> int:
272+
"""Return the memory usage of the internal buffers of the given operator."""
273+
return self._mem_op_internal[op]
274+
275+
def get_mem_op_outputs(self, op: PhysicalOperator) -> int:
276+
"""Return the memory usage of the outputs of the given operator."""
277+
return self._mem_op_outputs[op]
278+
271279
def get_op_usage_str(self, op: PhysicalOperator, *, verbose: bool) -> str:
272280
"""Return a human-readable string representation of the resource usage of
273281
the given operator."""
@@ -286,8 +294,8 @@ def get_op_usage_str(self, op: PhysicalOperator, *, verbose: bool) -> str:
286294

287295
if verbose:
288296
usage_str += (
289-
f" (in={memory_string(self._mem_op_internal[op])},"
290-
f"out={memory_string(self._mem_op_outputs[op])})"
297+
f" (in={memory_string(self.get_mem_op_internal(op))},"
298+
f"out={memory_string(self.get_mem_op_outputs(op))})"
291299
)
292300
if self._op_resource_allocator is not None:
293301
allocation = self._op_resource_allocator.get_allocation(op)
@@ -394,9 +402,12 @@ def get_op_outputs_object_store_usage_with_downstream(
394402
)
395403
return op_outputs_usage
396404

397-
def get_op_internal_object_store_usage(self, op: PhysicalOperator) -> int:
398-
"""Get the internal object store memory usage of the given operator"""
399-
return self._mem_op_internal[op]
405+
def has_materializing_downstream_op(self, op: PhysicalOperator) -> bool:
406+
"""Check if the operator has a downstream materializing operator."""
407+
return any(
408+
isinstance(next_op, MATERIALIZING_OPERATORS)
409+
for next_op in op.output_dependencies
410+
)
400411

401412

402413
def _get_first_pending_shuffle_op(topology: "Topology") -> int:
@@ -832,9 +843,7 @@ def update_budgets(
832843
op_mem_usage = 0
833844
# Add the memory usage of the operator itself,
834845
# excluding `_reserved_for_op_outputs`.
835-
op_mem_usage += self._resource_manager.get_op_internal_object_store_usage(
836-
op
837-
)
846+
op_mem_usage += self._resource_manager.get_mem_op_internal(op)
838847
# Add the portion of op outputs usage that has
839848
# exceeded `_reserved_for_op_outputs`.
840849
op_outputs_usage = self._resource_manager.get_op_outputs_object_store_usage_with_downstream(

python/ray/data/context.py

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -251,6 +251,7 @@ class ShuffleStrategy(str, enum.Enum):
251251
)
252252

253253

254+
# Dynamic output queue size backpressure disabled by default.
254255
DEFAULT_ENABLE_DYNAMIC_OUTPUT_QUEUE_SIZE_BACKPRESSURE: bool = env_bool(
255256
"RAY_DATA_ENABLE_DYNAMIC_OUTPUT_QUEUE_SIZE_BACKPRESSURE", False
256257
)

python/ray/data/tests/test_backpressure_policies.py

Lines changed: 91 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -60,10 +60,16 @@ def test_basic(self):
6060
map_op_no_concurrency: MagicMock(),
6161
}
6262

63+
mock_resource_manager = MagicMock()
64+
# Return None to skip dynamic output queue size backpressure check
65+
mock_resource_manager.get_op_usage.return_value = None
66+
mock_resource_manager.get_budget.return_value = None
67+
mock_resource_manager.is_op_eligible.return_value = False
68+
6369
policy = ConcurrencyCapBackpressurePolicy(
6470
DataContext.get_current(),
6571
topology,
66-
MagicMock(),
72+
mock_resource_manager,
6773
)
6874

6975
self.assertEqual(policy._concurrency_caps[map_op], concurrency)
@@ -177,6 +183,67 @@ def test_can_add_input_with_non_map_operator(self):
177183
# InputDataBuffer has infinite concurrency cap, so should always allow
178184
self.assertTrue(policy.can_add_input(input_op))
179185

186+
def test_can_add_input_with_ineligible_op(self):
187+
"""Test can_add_input when op is not eligible for backpressure."""
188+
input_op = InputDataBuffer(DataContext.get_current(), input_data=[MagicMock()])
189+
map_op = TaskPoolMapOperator(
190+
map_transformer=MagicMock(),
191+
data_context=DataContext.get_current(),
192+
input_op=input_op,
193+
max_concurrency=5,
194+
)
195+
map_op.metrics.num_tasks_running = 3
196+
197+
topology = {map_op: MagicMock(), input_op: MagicMock()}
198+
199+
mock_resource_manager = MagicMock()
200+
# Op is not eligible for backpressure
201+
mock_resource_manager.is_op_eligible.return_value = False
202+
203+
policy = ConcurrencyCapBackpressurePolicy(
204+
DataContext.get_current(),
205+
topology,
206+
mock_resource_manager,
207+
)
208+
policy.enable_dynamic_output_queue_size_backpressure = True
209+
210+
# Should skip dynamic backpressure and use basic cap check
211+
self.assertTrue(policy.can_add_input(map_op)) # 3 < 5
212+
213+
map_op.metrics.num_tasks_running = 5
214+
self.assertFalse(policy.can_add_input(map_op)) # 5 >= 5
215+
216+
def test_can_add_input_with_materializing_downstream_op(self):
217+
"""Test can_add_input when downstream op is a materializing operator."""
218+
input_op = InputDataBuffer(DataContext.get_current(), input_data=[MagicMock()])
219+
map_op = TaskPoolMapOperator(
220+
map_transformer=MagicMock(),
221+
data_context=DataContext.get_current(),
222+
input_op=input_op,
223+
max_concurrency=5,
224+
)
225+
map_op.metrics.num_tasks_running = 3
226+
227+
topology = {map_op: MagicMock(), input_op: MagicMock()}
228+
229+
mock_resource_manager = MagicMock()
230+
mock_resource_manager.is_op_eligible.return_value = True
231+
mock_resource_manager.has_materializing_downstream_op.return_value = True
232+
233+
policy = ConcurrencyCapBackpressurePolicy(
234+
DataContext.get_current(),
235+
topology,
236+
mock_resource_manager,
237+
)
238+
policy.enable_dynamic_output_queue_size_backpressure = True
239+
240+
# Should skip dynamic backpressure and use basic cap check
241+
# to avoid starving materializing operators
242+
self.assertTrue(policy.can_add_input(map_op)) # 3 < 5
243+
244+
map_op.metrics.num_tasks_running = 5
245+
self.assertFalse(policy.can_add_input(map_op)) # 5 >= 5
246+
180247
def test_can_add_input_with_object_store_memory_usage_ratio_above_threshold(self):
181248
"""Test can_add_input when object store memory usage ratio is above threshold."""
182249
input_op = InputDataBuffer(DataContext.get_current(), input_data=[MagicMock()])
@@ -193,8 +260,10 @@ def test_can_add_input_with_object_store_memory_usage_ratio_above_threshold(self
193260
mock_resource_manager = MagicMock()
194261

195262
# Mock object store memory usage ratio above threshold
196-
# Ratio = budget / (usage + budget) > OBJECT_STORE_BUDGET_RATIO
197-
threshold = ConcurrencyCapBackpressurePolicy.OBJECT_STORE_BUDGET_RATIO
263+
# Ratio = budget / (usage + budget) > AVAILABLE_OBJECT_STORE_BUDGET_THRESHOLD
264+
threshold = (
265+
ConcurrencyCapBackpressurePolicy.AVAILABLE_OBJECT_STORE_BUDGET_THRESHOLD
266+
)
198267
mock_usage = MagicMock()
199268
mock_usage.object_store_memory = 1000 # usage
200269
mock_budget = MagicMock()
@@ -207,6 +276,8 @@ def test_can_add_input_with_object_store_memory_usage_ratio_above_threshold(self
207276

208277
mock_resource_manager.get_op_usage.return_value = mock_usage
209278
mock_resource_manager.get_budget.return_value = mock_budget
279+
mock_resource_manager.is_op_eligible.return_value = True
280+
mock_resource_manager.has_materializing_downstream_op.return_value = False
210281

211282
policy = ConcurrencyCapBackpressurePolicy(
212283
DataContext.get_current(),
@@ -249,8 +320,10 @@ def test_can_add_input_with_object_store_memory_usage_ratio_below_threshold(self
249320
mock_resource_manager = MagicMock()
250321

251322
# Mock object store memory usage ratio below threshold
252-
# Ratio = budget / (usage + budget) < OBJECT_STORE_BUDGET_RATIO
253-
threshold = ConcurrencyCapBackpressurePolicy.OBJECT_STORE_BUDGET_RATIO
323+
# Ratio = budget / (usage + budget) < AVAILABLE_OBJECT_STORE_BUDGET_THRESHOLD
324+
threshold = (
325+
ConcurrencyCapBackpressurePolicy.AVAILABLE_OBJECT_STORE_BUDGET_THRESHOLD
326+
)
254327
mock_usage = MagicMock()
255328
mock_usage.object_store_memory = 1000 # usage
256329
mock_budget = MagicMock()
@@ -263,9 +336,11 @@ def test_can_add_input_with_object_store_memory_usage_ratio_below_threshold(self
263336

264337
mock_resource_manager.get_op_usage.return_value = mock_usage
265338
mock_resource_manager.get_budget.return_value = mock_budget
339+
mock_resource_manager.is_op_eligible.return_value = True
340+
mock_resource_manager.has_materializing_downstream_op.return_value = False
266341

267342
# Mock queue size methods
268-
mock_resource_manager.get_op_internal_object_store_usage.return_value = 100
343+
mock_resource_manager.get_mem_op_internal.return_value = 100
269344
mock_resource_manager.get_op_outputs_object_store_usage_with_downstream.return_value = (
270345
200
271346
)
@@ -286,9 +361,10 @@ def test_can_add_input_with_object_store_memory_usage_ratio_below_threshold(self
286361
policy._q_level_dev[map_op] = initial_dev
287362

288363
result = policy.can_add_input(map_op)
289-
# With queue size 300, initial level=200, dev=50, bounds=[100, 300]
290-
# Queue size 300 is at the upper bound, so should hold.
291-
# running=3 < effective_cap=3 should be False
364+
# With queue size 300, initial level=200, dev=50, bounds=[150, 250]
365+
# Queue size 300 is above the upper bound, so should backoff.
366+
# running=3, backoff by 1 -> effective_cap=2
367+
# running=3 < effective_cap=2 should be False
292368
self.assertFalse(result)
293369
# EWMA state should be updated when ratio < threshold
294370
# Level should move toward 300 (queue size)
@@ -310,7 +386,9 @@ def test_can_add_input_effective_cap_calculation(self):
310386
topology = {map_op: MagicMock(), input_op: MagicMock()}
311387

312388
mock_resource_manager = MagicMock()
313-
threshold = ConcurrencyCapBackpressurePolicy.OBJECT_STORE_BUDGET_RATIO
389+
threshold = (
390+
ConcurrencyCapBackpressurePolicy.AVAILABLE_OBJECT_STORE_BUDGET_THRESHOLD
391+
)
314392
mock_usage = MagicMock()
315393
mock_usage.object_store_memory = 1000
316394
mock_budget = MagicMock()
@@ -323,6 +401,8 @@ def test_can_add_input_effective_cap_calculation(self):
323401

324402
mock_resource_manager.get_op_usage.return_value = mock_usage
325403
mock_resource_manager.get_budget.return_value = mock_budget
404+
mock_resource_manager.is_op_eligible.return_value = True
405+
mock_resource_manager.has_materializing_downstream_op.return_value = False
326406

327407
policy = ConcurrencyCapBackpressurePolicy(
328408
DataContext.get_current(),
@@ -369,9 +449,7 @@ def test_can_add_input_effective_cap_calculation(self):
369449
description,
370450
) in test_cases:
371451
with self.subTest(description=description):
372-
mock_resource_manager.get_op_internal_object_store_usage.return_value = (
373-
internal_usage
374-
)
452+
mock_resource_manager.get_mem_op_internal.return_value = internal_usage
375453
mock_resource_manager.get_op_outputs_object_store_usage_with_downstream.return_value = (
376454
downstream_usage
377455
)

0 commit comments

Comments
 (0)