11import logging
2- from typing import TYPE_CHECKING
2+ from typing import TYPE_CHECKING , Optional
33
44from .backpressure_policy import BackpressurePolicy
5- from ray .data ._internal .execution .operators .actor_pool_map_operator import (
6- ActorPoolMapOperator ,
7- )
5+ from ray ._private .ray_constants import env_float
86from ray .data .context import DataContext
97
108if TYPE_CHECKING :
2018class DownstreamCapacityBackpressurePolicy (BackpressurePolicy ):
2119 """Backpressure policy based on downstream processing capacity.
2220
23- This policy triggers backpressure when the output bundles size exceeds both:
24- 1. A ratio threshold multiplied by the number of running tasks in downstream operators
25- 2. An absolute threshold for the output bundles size
26-
27- The policy monitors actual downstream processing capacity by tracking the number
28- of currently running tasks rather than configured parallelism. This approach
29- ensures effective backpressure even when cluster resources are insufficient or
30- scaling is slow, preventing memory pressure and maintaining pipeline stability.
31-
32- Key benefits:
33- - Prevents memory bloat from unprocessed output objects
34- - Adapts to actual cluster conditions and resource availability
35- - Maintains balanced throughput across pipeline operators
36- - Reduces object spilling and unnecessary rebuilds
21+ To backpressure a given operator, use queue size build up / downstream capacity ratio.
22+ This ratio represents the upper limit of buffering in object store between pipeline stages
23+ to optimize for throughput.
3724 """
3825
26+ # Threshold for per-Op object store budget utilization vs total
27+ # (utilization / total) ratio to enable downstream capacity backpressure.
28+ OBJECT_STORE_BUDGET_UTIL_THRESHOLD = env_float (
29+ "RAY_DATA_DOWNSTREAM_CAPACITY_OBJECT_STORE_BUDGET_UTIL_THRESHOLD" , 0.9
30+ )
31+
3932 @property
4033 def name (self ) -> str :
4134 return "DownstreamCapacity"
@@ -47,50 +40,113 @@ def __init__(
4740 resource_manager : "ResourceManager" ,
4841 ):
4942 super ().__init__ (data_context , topology , resource_manager )
50- self ._backpressure_concurrency_ratio = (
43+ self ._backpressure_capacity_ratio = (
5144 self ._data_context .downstream_capacity_backpressure_ratio
5245 )
53- self ._backpressure_max_queued_blocks = (
54- self ._data_context .downstream_capacity_backpressure_max_queued_bundles
55- )
56- self ._backpressure_disabled = (
57- self ._backpressure_concurrency_ratio is None
58- or self ._backpressure_max_queued_blocks is None
46+ if self ._backpressure_capacity_ratio is not None :
47+ logger .debug (
48+ f"DownstreamCapacityBackpressurePolicy enabled with backpressure capacity ratio: { self ._backpressure_capacity_ratio } "
49+ )
50+
51+ def _get_queue_size_bytes (self , op : "PhysicalOperator" ) -> int :
52+ """Get the output current queue size
53+ (this operator + ineligible downstream operators) in bytes for the given operator.
54+ """
55+ op_outputs_usage = self ._topology [op ].output_queue_bytes ()
56+ # Also account the downstream ineligible operators' memory usage.
57+ op_outputs_usage += sum (
58+ self ._resource_manager .get_op_usage (next_op ).object_store_memory
59+ for next_op in self ._resource_manager ._get_downstream_ineligible_ops (op )
5960 )
61+ return op_outputs_usage
6062
61- def _max_concurrent_tasks (self , op : "PhysicalOperator" ) -> int :
62- if isinstance (op , ActorPoolMapOperator ):
63- return sum (
64- [
65- actor_pool .max_concurrent_tasks ()
66- for actor_pool in op .get_autoscaling_actor_pools ()
67- ]
68- )
69- return op .num_active_tasks ()
63+ def _get_downstream_capacity_size_bytes (self , op : "PhysicalOperator" ) -> int :
64+ """Get the downstream capacity size for the given operator.
7065
71- def can_add_input (self , op : "PhysicalOperator" ) -> bool :
72- """Determine if we can add input to the operator based on downstream capacity."""
73- if self ._backpressure_disabled :
74- return True
66+ Downstream capacity size is the sum of the pending task inputs of the
67+ downstream eligible operators.
68+
69+ If an output dependency is ineligible, skip it and recurse down to find
70+ eligible output dependencies. If there are no output dependencies,
71+ return external consumer bytes.
72+ """
73+ if not op .output_dependencies :
74+ # No output dependencies, return external consumer bytes.
75+ return self ._resource_manager .get_external_consumer_bytes ()
76+
77+ total_capacity_size_bytes = 0
7578 for output_dependency in op .output_dependencies :
76- total_enqueued_blocks = self ._topology [
77- output_dependency
78- ].total_enqueued_input_blocks ()
79+ if self ._resource_manager .is_op_eligible (output_dependency ):
80+ # Output dependency is eligible, add its pending task inputs.
81+ total_capacity_size_bytes += (
82+ output_dependency .metrics .obj_store_mem_pending_task_inputs or 0
83+ )
84+ else :
85+ # Output dependency is ineligible, recurse down to find eligible ops.
86+ total_capacity_size_bytes += self ._get_downstream_capacity_size_bytes (
87+ output_dependency
88+ )
89+ return total_capacity_size_bytes
7990
80- avg_inputs_per_task = (
81- output_dependency .metrics .num_task_inputs_processed
82- / max (output_dependency .metrics .num_tasks_finished , 1 )
83- )
84- outstanding_tasks = total_enqueued_blocks / max (avg_inputs_per_task , 1 )
85- max_allowed_outstanding = (
86- self ._max_concurrent_tasks (output_dependency )
87- * self ._backpressure_concurrency_ratio
88- )
91+ def _should_skip_backpressure (self , op : "PhysicalOperator" ) -> bool :
92+ """Check if backpressure should be skipped for the operator.
93+ TODO(srinathk10): Extract this to common logic to skip invoking BackpressurePolicy.
94+ """
95+ if self ._backpressure_capacity_ratio is None :
96+ # Downstream capacity backpressure is disabled.
97+ return True
98+ if not self ._resource_manager .is_op_eligible (op ):
99+ # Operator is not eligible for backpressure.
100+ return True
101+ if self ._resource_manager .is_materializing_op (op ):
102+ # Operator is materializing, so no need to perform backpressure.
103+ return True
104+ if self ._resource_manager .has_materializing_downstream_op (op ):
105+ # Downstream operator is materializing, so can't perform backpressure
106+ # based on downstream capacity which requires full materialization.
107+ return True
108+ return False
109+
110+ def _get_queue_ratio (self , op : "PhysicalOperator" ) -> float :
111+ """Get queue/capacity ratio for the operator."""
112+ queue_size_bytes = self ._get_queue_size_bytes (op )
113+ downstream_capacity_size_bytes = self ._get_downstream_capacity_size_bytes (op )
114+ if downstream_capacity_size_bytes == 0 :
115+ # No downstream capacity to backpressure against, so no backpressure.
116+ return 0
117+ return queue_size_bytes / downstream_capacity_size_bytes
89118
90- if (
91- total_enqueued_blocks > self ._backpressure_max_queued_blocks
92- and outstanding_tasks > max_allowed_outstanding
93- ):
94- return False
119+ def _should_apply_backpressure (self , op : "PhysicalOperator" ) -> bool :
120+ """Check if backpressure should be applied for the operator.
121+
122+ Returns True if backpressure should be applied, False otherwise.
123+ """
124+ if self ._should_skip_backpressure (op ):
125+ return False
126+
127+ utilized_budget_fraction = (
128+ self ._resource_manager .get_utilized_object_store_budget_fraction (op )
129+ )
130+ if (
131+ utilized_budget_fraction is not None
132+ and utilized_budget_fraction <= self .OBJECT_STORE_BUDGET_UTIL_THRESHOLD
133+ ):
134+ # Utilized budget fraction is below threshold, so should skip backpressure.
135+ return False
136+
137+ queue_ratio = self ._get_queue_ratio (op )
138+ # Apply backpressure if queue ratio exceeds the threshold.
139+ return queue_ratio > self ._backpressure_capacity_ratio
140+
141+ def can_add_input (self , op : "PhysicalOperator" ) -> bool :
142+ """Determine if we can add input to the operator based on
143+ downstream capacity.
144+ """
145+ return not self ._should_apply_backpressure (op )
95146
96- return True
147+ def max_task_output_bytes_to_read (self , op : "PhysicalOperator" ) -> Optional [int ]:
148+ """Return the maximum bytes of pending task outputs can be read for
149+ the given operator. None means no limit."""
150+ if self ._should_apply_backpressure (op ):
151+ return 0
152+ return None
0 commit comments