-
Notifications
You must be signed in to change notification settings - Fork 1.8k
fix: synchronize partition bounds reporting in HashJoin #17452
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
Changes from 2 commits
fb0a153
02053aa
ef2c773
35da826
c24ef79
2a41ced
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 |
|---|---|---|
|
|
@@ -19,6 +19,8 @@ | |
| // TODO: include the link to the Dynamic Filter blog post. | ||
|
|
||
| use std::fmt; | ||
| use std::future::Future; | ||
| use std::sync::atomic::AtomicUsize; | ||
| use std::sync::Arc; | ||
|
|
||
| use crate::joins::PartitionMode; | ||
|
|
@@ -30,6 +32,7 @@ use datafusion_expr::Operator; | |
| use datafusion_physical_expr::expressions::{lit, BinaryExpr, DynamicFilterPhysicalExpr}; | ||
| use datafusion_physical_expr::{PhysicalExpr, PhysicalExprRef}; | ||
|
|
||
| use futures::task::AtomicWaker; | ||
| use itertools::Itertools; | ||
| use parking_lot::Mutex; | ||
|
|
||
|
|
@@ -119,7 +122,9 @@ struct SharedBoundsState { | |
| /// Each element represents the column bounds computed by one partition. | ||
| bounds: Vec<PartitionBounds>, | ||
| /// Number of partitions that have reported completion. | ||
| completed_partitions: usize, | ||
| completed_partitions: Arc<AtomicUsize>, | ||
| /// Cached wakers to wake when all partitions are complete | ||
| wakers: Vec<Arc<AtomicWaker>>, | ||
| } | ||
|
|
||
| impl SharedBoundsAccumulator { | ||
|
|
@@ -170,7 +175,8 @@ impl SharedBoundsAccumulator { | |
| Self { | ||
| inner: Mutex::new(SharedBoundsState { | ||
| bounds: Vec::with_capacity(expected_calls), | ||
| completed_partitions: 0, | ||
| completed_partitions: Arc::new(AtomicUsize::new(0)), | ||
| wakers: Vec::new(), | ||
| }), | ||
| total_partitions: expected_calls, | ||
| dynamic_filter, | ||
|
|
@@ -253,39 +259,69 @@ impl SharedBoundsAccumulator { | |
| /// bounds from the current partition, increments the completion counter, and when all | ||
| /// partitions have reported, creates an OR'd filter from individual partition bounds. | ||
| /// | ||
| /// It returns a [`BoundsWaiter`] future that can be awaited to ensure the filter has been | ||
| /// updated before proceeding. This is important to delay probe-side scans until the filter | ||
| /// is ready. | ||
| /// | ||
| /// # Arguments | ||
| /// * `partition` - The partition identifier reporting its bounds | ||
| /// * `partition_bounds` - The bounds computed by this partition (if any) | ||
| /// | ||
| /// # Returns | ||
| /// * `Result<()>` - Ok if successful, Err if filter update failed | ||
| /// * `Result<Option<BoundsWaiter>>` - Ok if successful, Err if filter update failed | ||
| pub(crate) fn report_partition_bounds( | ||
| &self, | ||
| partition: usize, | ||
| partition_bounds: Option<Vec<ColumnBounds>>, | ||
| ) -> Result<()> { | ||
| let mut inner = self.inner.lock(); | ||
| ) -> Result<Option<BoundsWaiter>> { | ||
| // Scope for lock to avoid holding it across await points | ||
| let maybe_waiter = { | ||
| let mut inner = self.inner.lock(); | ||
|
|
||
| // Store bounds in the accumulator - this runs once per partition | ||
| if let Some(bounds) = partition_bounds { | ||
| // Only push actual bounds if they exist | ||
| inner.bounds.push(PartitionBounds::new(partition, bounds)); | ||
| } | ||
| // Store bounds in the accumulator - this runs once per partition | ||
| if let Some(bounds) = partition_bounds { | ||
| // Only push actual bounds if they exist | ||
| inner.bounds.push(PartitionBounds::new(partition, bounds)); | ||
| } | ||
|
|
||
| // Increment the completion counter | ||
| // Even empty partitions must report to ensure proper termination | ||
| inner.completed_partitions += 1; | ||
| let completed = inner.completed_partitions; | ||
| let total_partitions = self.total_partitions; | ||
|
|
||
| // Critical synchronization point: Only update the filter when ALL partitions are complete | ||
| // Troubleshooting: If you see "completed > total_partitions", check partition | ||
| // count calculation in new_from_partition_mode() - it may not match actual execution calls | ||
| if completed == total_partitions && !inner.bounds.is_empty() { | ||
| let filter_expr = self.create_filter_from_partition_bounds(&inner.bounds)?; | ||
| self.dynamic_filter.update(filter_expr)?; | ||
| } | ||
| // Increment the completion counter | ||
| // Even empty partitions must report to ensure proper termination | ||
| inner | ||
| .completed_partitions | ||
| .fetch_add(1, std::sync::atomic::Ordering::SeqCst); | ||
| let completed = inner | ||
| .completed_partitions | ||
| .load(std::sync::atomic::Ordering::SeqCst); | ||
| let total_partitions = self.total_partitions; | ||
|
|
||
| // Critical synchronization point: Only update the filter when ALL partitions are complete | ||
| // Troubleshooting: If you see "completed > total_partitions", check partition | ||
| // count calculation in new_from_partition_mode() - it may not match actual execution calls | ||
| if completed == total_partitions { | ||
| if !inner.bounds.is_empty() { | ||
| let filter_expr = | ||
| self.create_filter_from_partition_bounds(&inner.bounds)?; | ||
| self.dynamic_filter.update(filter_expr)?; | ||
| } | ||
|
|
||
| // Notify any waiters that the filter is ready | ||
| for waker in inner.wakers.drain(..) { | ||
| waker.wake(); | ||
| } | ||
|
|
||
| None | ||
| } else { | ||
| let waker = Arc::new(AtomicWaker::new()); | ||
| inner.wakers.push(Arc::clone(&waker)); | ||
| Some(BoundsWaiter::new( | ||
| total_partitions, | ||
| Arc::clone(&inner.completed_partitions), | ||
| waker, | ||
| )) | ||
| } | ||
| }; | ||
|
|
||
| Ok(()) | ||
| Ok(maybe_waiter) | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -294,3 +330,48 @@ impl fmt::Debug for SharedBoundsAccumulator { | |
| write!(f, "SharedBoundsAccumulator") | ||
| } | ||
| } | ||
|
|
||
| /// Utility future to wait until all partitions have reported completion | ||
|
||
| /// and the dynamic filter has been updated. | ||
| #[derive(Clone)] | ||
| pub(crate) struct BoundsWaiter { | ||
| waker: Arc<AtomicWaker>, | ||
| total: usize, | ||
| completed: Arc<AtomicUsize>, | ||
| } | ||
|
|
||
| impl BoundsWaiter { | ||
| pub fn new( | ||
| total: usize, | ||
| completed: Arc<AtomicUsize>, | ||
| waker: Arc<AtomicWaker>, | ||
| ) -> Self { | ||
| Self { | ||
| waker, | ||
| total, | ||
| completed, | ||
| } | ||
| } | ||
| } | ||
|
|
||
| impl Future for BoundsWaiter { | ||
| type Output = (); | ||
|
|
||
| fn poll( | ||
| self: std::pin::Pin<&mut Self>, | ||
| cx: &mut std::task::Context<'_>, | ||
| ) -> std::task::Poll<Self::Output> { | ||
| // Quick check to avoid registration if already complete | ||
| if self.completed.load(std::sync::atomic::Ordering::Relaxed) >= self.total { | ||
| return std::task::Poll::Ready(()); | ||
| } | ||
|
|
||
| self.waker.register(cx.waker()); | ||
|
|
||
| if self.completed.load(std::sync::atomic::Ordering::Relaxed) >= self.total { | ||
| std::task::Poll::Ready(()) | ||
| } else { | ||
| std::task::Poll::Pending | ||
| } | ||
| } | ||
| } | ||
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.
Will take this out, just leaving here for now for visualization.
I wonder what are good ways to regression test this 🤔
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.
The best I can think of would be to make a custom ExecutionPlan that delays partition 0 for 1s or something and check that we still read the expected number of rows on the probe side
Uh oh!
There was an error while loading. Please reload this page.
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.
Hey @adriangb, I modified the existing partitioned test to inspect metrics from the probe side scan and verify rows are correctly being filtered out there. I think we can get away without the delay since I applied the patch to main and the test became flaky, as expected. Does that sound good to you or did you have something else in mind?