@@ -32,6 +32,7 @@ use datafusion_physical_expr::{PhysicalExpr, PhysicalExprRef};
32
32
33
33
use itertools:: Itertools ;
34
34
use parking_lot:: Mutex ;
35
+ use tokio:: sync:: Barrier ;
35
36
36
37
/// Represents the minimum and maximum values for a specific column.
37
38
/// Used in dynamic filter pushdown to establish value boundaries.
@@ -86,9 +87,9 @@ impl PartitionBounds {
86
87
/// ## Synchronization Strategy
87
88
///
88
89
/// 1. Each partition computes bounds from its build-side data
89
- /// 2. Bounds are stored in the shared HashMap (indexed by partition_id)
90
- /// 3. A counter tracks how many partitions have reported their bounds
91
- /// 4. When the last partition reports (completed == total) , bounds are merged and filter is updated
90
+ /// 2. Bounds are stored in the shared vector
91
+ /// 3. A barrier tracks how many partitions have reported their bounds
92
+ /// 4. When the last partition reports, bounds are merged and the filter is updated exactly once
92
93
///
93
94
/// ## Partition Counting
94
95
///
@@ -103,10 +104,7 @@ impl PartitionBounds {
103
104
pub ( crate ) struct SharedBoundsAccumulator {
104
105
/// Shared state protected by a single mutex to avoid ordering concerns
105
106
inner : Mutex < SharedBoundsState > ,
106
- /// Total number of partitions.
107
- /// Need to know this so that we can update the dynamic filter once we are done
108
- /// building *all* of the hash tables.
109
- total_partitions : usize ,
107
+ barrier : Barrier ,
110
108
/// Dynamic filter for pushdown to probe side
111
109
dynamic_filter : Arc < DynamicFilterPhysicalExpr > ,
112
110
/// Right side join expressions needed for creating filter bounds
@@ -118,8 +116,6 @@ struct SharedBoundsState {
118
116
/// Bounds from completed partitions.
119
117
/// Each element represents the column bounds computed by one partition.
120
118
bounds : Vec < PartitionBounds > ,
121
- /// Number of partitions that have reported completion.
122
- completed_partitions : usize ,
123
119
}
124
120
125
121
impl SharedBoundsAccumulator {
@@ -170,9 +166,8 @@ impl SharedBoundsAccumulator {
170
166
Self {
171
167
inner : Mutex :: new ( SharedBoundsState {
172
168
bounds : Vec :: with_capacity ( expected_calls) ,
173
- completed_partitions : 0 ,
174
169
} ) ,
175
- total_partitions : expected_calls,
170
+ barrier : Barrier :: new ( expected_calls) ,
176
171
dynamic_filter,
177
172
on_right,
178
173
}
@@ -253,36 +248,44 @@ impl SharedBoundsAccumulator {
253
248
/// bounds from the current partition, increments the completion counter, and when all
254
249
/// partitions have reported, creates an OR'd filter from individual partition bounds.
255
250
///
251
+ /// This method is async and uses a [`tokio::sync::Barrier`] to wait for all partitions
252
+ /// to report their bounds. Once that occurs, the method will resolve for all callers and the
253
+ /// dynamic filter will be updated exactly once.
254
+ ///
255
+ /// # Note
256
+ ///
257
+ /// As barriers are reusable, it is likely an error to call this method more times than the
258
+ /// total number of partitions - as it can lead to pending futures that never resolve. We rely
259
+ /// on correct usage from the caller rather than imposing additional checks here. If this is a concern,
260
+ /// consider making the resulting future shared so the ready result can be reused.
261
+ ///
256
262
/// # Arguments
263
+ /// * `partition` - The partition identifier reporting its bounds
257
264
/// * `partition_bounds` - The bounds computed by this partition (if any)
258
265
///
259
266
/// # Returns
260
267
/// * `Result<()>` - Ok if successful, Err if filter update failed
261
- pub ( crate ) fn report_partition_bounds (
268
+ pub ( crate ) async fn report_partition_bounds (
262
269
& self ,
263
270
partition : usize ,
264
271
partition_bounds : Option < Vec < ColumnBounds > > ,
265
272
) -> Result < ( ) > {
266
- let mut inner = self . inner . lock ( ) ;
267
-
268
273
// Store bounds in the accumulator - this runs once per partition
269
274
if let Some ( bounds) = partition_bounds {
270
- // Only push actual bounds if they exist
271
- inner. bounds . push ( PartitionBounds :: new ( partition, bounds) ) ;
275
+ self . inner
276
+ . lock ( )
277
+ . bounds
278
+ . push ( PartitionBounds :: new ( partition, bounds) ) ;
272
279
}
273
280
274
- // Increment the completion counter
275
- // Even empty partitions must report to ensure proper termination
276
- inner. completed_partitions += 1 ;
277
- let completed = inner. completed_partitions ;
278
- let total_partitions = self . total_partitions ;
279
-
280
- // Critical synchronization point: Only update the filter when ALL partitions are complete
281
- // Troubleshooting: If you see "completed > total_partitions", check partition
282
- // count calculation in new_from_partition_mode() - it may not match actual execution calls
283
- if completed == total_partitions && !inner. bounds . is_empty ( ) {
284
- let filter_expr = self . create_filter_from_partition_bounds ( & inner. bounds ) ?;
285
- self . dynamic_filter . update ( filter_expr) ?;
281
+ if self . barrier . wait ( ) . await . is_leader ( ) {
282
+ // All partitions have reported, so we can update the filter
283
+ let inner = self . inner . lock ( ) ;
284
+ if !inner. bounds . is_empty ( ) {
285
+ let filter_expr =
286
+ self . create_filter_from_partition_bounds ( & inner. bounds ) ?;
287
+ self . dynamic_filter . update ( filter_expr) ?;
288
+ }
286
289
}
287
290
288
291
Ok ( ( ) )
0 commit comments