Add hook for sharing join state in distributed execution#12523
Add hook for sharing join state in distributed execution#12523thinkharderdev wants to merge 1 commit intoapache:mainfrom
Conversation
|
Set as draft for now as I still need to integrate into |
| let reservation = | ||
| MemoryConsumer::new("HashJoinInput").register(context.memory_pool()); | ||
|
|
||
| let probe_threads = shared_state |
There was a problem hiding this comment.
If shared state is available we use the num_task_partitions instead of output partitioning to determine the number of local probe threads
| return Poll::Ready(Ok(StatefulStreamResult::Continue)); | ||
| } | ||
|
|
||
| if let Some(shared_state) = build_side.left_data.shared_state.as_ref() { |
There was a problem hiding this comment.
When all local probe threads are complete and there is shared state we need to probe that before competing
| assert_batches_sorted_eq!(expected, &batches); | ||
| } | ||
|
|
||
| struct Coordinator { |
There was a problem hiding this comment.
Usage example here. In a real-world scenario the Coordinator would likely be an external service which we communicate with through an rpc call
| use parking_lot::Mutex; | ||
|
|
||
| /// `SharedJoinState` provides an extension point allowing | ||
| /// `HashJoinStream` to share the `visited_indices_bitmap` of the build side of a join |
There was a problem hiding this comment.
is it left side or right side indices?
|
@korowa FYI |
| /// across probe tasks without shared memory. | ||
| /// | ||
| /// This can be used to, for example, implement a left outer join efficiently as a broadcast join | ||
| /// if the left side is small |
There was a problem hiding this comment.
is it left side small or right? My feeling was the left(driving) table is huge and right is small
There was a problem hiding this comment.
No, it's the opposite. The left (build) side is small and can be efficiently broadcast. Then the right (probe) side can be partitioned across multiple nodes with the build side broadcast to all of them.
| } | ||
| } | ||
|
|
||
| fn merge_bitmap(m1: &mut BooleanBufferBuilder, m2: BooleanBuffer) -> Result<()> { |
There was a problem hiding this comment.
bitmap here is boolean bitmask of what was visited or matched by join/filter?
There was a problem hiding this comment.
Correct. What will be shared and merged is the JoinLeftData::visited_indices_bitmap
|
Maybe its easier to build some diagram in draw.io or something? |
Yeah, good idea I'll work something up |
|
Thank you for your contribution. Unfortunately, this pull request is stale because it has been open 60 days with no activity. Please remove the stale label or comment or this will be closed in 7 days. |
|
Thank you for your contribution. Unfortunately, this pull request is stale because it has been open 60 days with no activity. Please remove the stale label or comment or this will be closed in 7 days. |
|
Thank you for your contribution. Unfortunately, this pull request is stale because it has been open 60 days with no activity. Please remove the stale label or comment or this will be closed in 7 days. |
|
Thank you for your contribution. Unfortunately, this pull request is stale because it has been open 60 days with no activity. Please remove the stale label or comment or this will be closed in 7 days. |
--- [Cherry-pick summary: v46→v47] Source commit: 76905a1 (Hook for doing distributed `CollectLeft` joins (#269/apache#12523)) Strategy: cherry-picked, minor adaptions (3 conflicts in hash_join.rs; v47 refactored execute() to use try_once instead of once, requiring Ok() wrapper; partition check preserved; stale None first-arg dropped; mod.rs re-export conflict resolved by keeping both PhysicalExprRef import and new pub use exports) Upstream PR: fork-only (apache#12523 ref) Test coverage: adequate Tests: cargo nextest run -p datafusion-physical-plan passed (899 tests) Co-Authored-By: Claude Sonnet 4.6 <noreply@anthropic.com>
--- [Cherry-pick summary: v46→v47] Source commit: 76905a1 (Hook for doing distributed `CollectLeft` joins (#269/apache#12523)) Strategy: cherry-picked, minor adaptions (3 conflicts in hash_join.rs; v47 refactored execute() to use try_once instead of once, requiring Ok() wrapper; partition check preserved; stale None first-arg dropped; mod.rs re-export conflict resolved by keeping both PhysicalExprRef import and new pub use exports) Upstream PR: fork-only (apache#12523 ref) Test coverage: adequate Tests: cargo nextest run -p datafusion-physical-plan passed (899 tests) Co-Authored-By: Claude Sonnet 4.6 <noreply@anthropic.com>
--- [Cherry-pick summary: v46→v47] Source commit: 76905a1 (Hook for doing distributed `CollectLeft` joins (#269/apache#12523)) Strategy: cherry-picked, minor adaptions (3 conflicts in hash_join.rs; v47 refactored execute() to use try_once instead of once, requiring Ok() wrapper; partition check preserved; stale None first-arg dropped; mod.rs re-export conflict resolved by keeping both PhysicalExprRef import and new pub use exports) Upstream PR: fork-only (apache#12523 ref) Test coverage: adequate Tests: cargo nextest run -p datafusion-physical-plan passed (899 tests) Co-Authored-By: Claude Sonnet 4.6 <noreply@anthropic.com>
--- [Cherry-pick summary: v46→v47] Source commit: 76905a1 (Hook for doing distributed `CollectLeft` joins (#269/apache#12523)) Strategy: cherry-picked, minor adaptions (3 conflicts in hash_join.rs; v47 refactored execute() to use try_once instead of once, requiring Ok() wrapper; partition check preserved; stale None first-arg dropped; mod.rs re-export conflict resolved by keeping both PhysicalExprRef import and new pub use exports) Upstream PR: fork-only (apache#12523 ref) Test coverage: adequate Tests: cargo nextest run -p datafusion-physical-plan passed (899 tests) Co-Authored-By: Claude Sonnet 4.6 <noreply@anthropic.com>
--- [Cherry-pick summary: v46→v47] Source commit: 76905a1 (Hook for doing distributed `CollectLeft` joins (#269/apache#12523)) Strategy: cherry-picked, minor adaptions (3 conflicts in hash_join.rs; v47 refactored execute() to use try_once instead of once, requiring Ok() wrapper; partition check preserved; stale None first-arg dropped; mod.rs re-export conflict resolved by keeping both PhysicalExprRef import and new pub use exports) Upstream PR: fork-only (apache#12523 ref) Test coverage: adequate Tests: cargo nextest run -p datafusion-physical-plan passed (899 tests) Co-Authored-By: Claude Sonnet 4.6 <noreply@anthropic.com>
Combines three fork-only commits from v49: - Hook for doing distributed CollectLeft joins (#269/apache#12523) - Add JoinContext with JoinLeftData to TaskContext in HashJoinExec (#300) - Make HASH_JOIN_SEED public (fork-only) Adds SharedJoinState/SharedJoinStateImpl trait for distributed probe coordination, JoinContext for sharing build-side state via TaskContext, contains_hash on JoinHashMapType, and converts process_unmatched_build_batch to async for shared state polling. Co-Authored-By: Claude Opus 4.6 <noreply@anthropic.com>
Combines three fork-only commits from v49: - Hook for doing distributed CollectLeft joins (#269/apache#12523) - Add JoinContext with JoinLeftData to TaskContext in HashJoinExec (#300) - Make HASH_JOIN_SEED public (fork-only) Adds SharedJoinState/SharedJoinStateImpl trait for distributed probe coordination, JoinContext for sharing build-side state via TaskContext, contains_hash on JoinHashMapType, and converts process_unmatched_build_batch to async for shared state polling. Co-Authored-By: Claude Opus 4.6 <noreply@anthropic.com>
Combines three fork-only commits from v49: - Hook for doing distributed CollectLeft joins (#269/apache#12523) - Add JoinContext with JoinLeftData to TaskContext in HashJoinExec (#300) - Make HASH_JOIN_SEED public (fork-only) Adds SharedJoinState/SharedJoinStateImpl trait for distributed probe coordination, JoinContext for sharing build-side state via TaskContext, contains_hash on JoinHashMapType, and converts process_unmatched_build_batch to async for shared state polling. Co-Authored-By: Claude Opus 4.6 <noreply@anthropic.com>
Which issue does this PR close?
Closes #12454
Rationale for this change
What changes are included in this PR?
Are these changes tested?
Are there any user-facing changes?