@@ -32,6 +32,7 @@ use datafusion_physical_expr::{PhysicalExpr, PhysicalExprRef};
3232
3333use itertools:: Itertools ;
3434use parking_lot:: Mutex ;
35+ use tokio:: sync:: Barrier ;
3536
3637/// Represents the minimum and maximum values for a specific column.
3738/// Used in dynamic filter pushdown to establish value boundaries.
@@ -86,9 +87,9 @@ impl PartitionBounds {
8687/// ## Synchronization Strategy
8788///
8889/// 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
9293///
9394/// ## Partition Counting
9495///
@@ -103,10 +104,7 @@ impl PartitionBounds {
103104pub ( crate ) struct SharedBoundsAccumulator {
104105 /// Shared state protected by a single mutex to avoid ordering concerns
105106 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 ,
110108 /// Dynamic filter for pushdown to probe side
111109 dynamic_filter : Arc < DynamicFilterPhysicalExpr > ,
112110 /// Right side join expressions needed for creating filter bounds
@@ -118,8 +116,6 @@ struct SharedBoundsState {
118116 /// Bounds from completed partitions.
119117 /// Each element represents the column bounds computed by one partition.
120118 bounds : Vec < PartitionBounds > ,
121- /// Number of partitions that have reported completion.
122- completed_partitions : usize ,
123119}
124120
125121impl SharedBoundsAccumulator {
@@ -170,9 +166,8 @@ impl SharedBoundsAccumulator {
170166 Self {
171167 inner : Mutex :: new ( SharedBoundsState {
172168 bounds : Vec :: with_capacity ( expected_calls) ,
173- completed_partitions : 0 ,
174169 } ) ,
175- total_partitions : expected_calls,
170+ barrier : Barrier :: new ( expected_calls) ,
176171 dynamic_filter,
177172 on_right,
178173 }
@@ -253,36 +248,44 @@ impl SharedBoundsAccumulator {
253248 /// bounds from the current partition, increments the completion counter, and when all
254249 /// partitions have reported, creates an OR'd filter from individual partition bounds.
255250 ///
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+ ///
256262 /// # Arguments
263+ /// * `partition` - The partition identifier reporting its bounds
257264 /// * `partition_bounds` - The bounds computed by this partition (if any)
258265 ///
259266 /// # Returns
260267 /// * `Result<()>` - Ok if successful, Err if filter update failed
261- pub ( crate ) fn report_partition_bounds (
268+ pub ( crate ) async fn report_partition_bounds (
262269 & self ,
263270 partition : usize ,
264271 partition_bounds : Option < Vec < ColumnBounds > > ,
265272 ) -> Result < ( ) > {
266- let mut inner = self . inner . lock ( ) ;
267-
268273 // Store bounds in the accumulator - this runs once per partition
269274 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) ) ;
272279 }
273280
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+ }
286289 }
287290
288291 Ok ( ( ) )
0 commit comments