Skip to content

Add hook for sharing join state in distributed execution#12523

Closed
thinkharderdev wants to merge 1 commit intoapache:mainfrom
thinkharderdev:ISSUE-12454
Closed

Add hook for sharing join state in distributed execution#12523
thinkharderdev wants to merge 1 commit intoapache:mainfrom
thinkharderdev:ISSUE-12454

Conversation

@thinkharderdev
Copy link
Copy Markdown
Contributor

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?

@github-actions github-actions Bot added the physical-expr Changes to the physical-expr crates label Sep 18, 2024
@thinkharderdev
Copy link
Copy Markdown
Contributor Author

Set as draft for now as I still need to integrate into NestedLoopJoinStream but the basic approach is all here

let reservation =
MemoryConsumer::new("HashJoinInput").register(context.memory_pool());

let probe_threads = shared_state
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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() {
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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 {
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

is it left side or right side indices?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Left

@comphead
Copy link
Copy Markdown
Contributor

@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
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

is it left side small or right? My feeling was the left(driving) table is huge and right is small

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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<()> {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

bitmap here is boolean bitmask of what was visited or matched by join/filter?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Correct. What will be shared and merged is the JoinLeftData::visited_indices_bitmap

@comphead
Copy link
Copy Markdown
Contributor

Maybe its easier to build some diagram in draw.io or something?
I got the point about shared state but I'm not sure how it will be travelling from caller side and to caller side

@thinkharderdev
Copy link
Copy Markdown
Contributor Author

Maybe its easier to build some diagram in draw.io or something? I got the point about shared state but I'm not sure how it will be travelling from caller side and to caller side

Yeah, good idea I'll work something up

@github-actions
Copy link
Copy Markdown

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.

@github-actions github-actions Bot added the Stale PR has not had any activity for some time label Nov 20, 2024
@Dandandan Dandandan removed the Stale PR has not had any activity for some time label Nov 20, 2024
@github-actions
Copy link
Copy Markdown

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.

@github-actions github-actions Bot added the Stale PR has not had any activity for some time label Jan 20, 2025
@github-actions github-actions Bot closed this Jan 27, 2025
@Dandandan Dandandan reopened this Jan 29, 2025
@github-actions github-actions Bot removed the Stale PR has not had any activity for some time label Jan 30, 2025
@github-actions
Copy link
Copy Markdown

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.

@github-actions github-actions Bot added the Stale PR has not had any activity for some time label Mar 31, 2025
@github-actions github-actions Bot closed this Apr 10, 2025
@Dandandan Dandandan reopened this Apr 14, 2025
@github-actions github-actions Bot removed the Stale PR has not had any activity for some time label Apr 15, 2025
@github-actions
Copy link
Copy Markdown

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.

@github-actions github-actions Bot added the Stale PR has not had any activity for some time label Jun 14, 2025
@github-actions github-actions Bot closed this Jun 21, 2025
avantgardnerio pushed a commit to coralogix/arrow-datafusion that referenced this pull request Dec 15, 2025
avantgardnerio pushed a commit to coralogix/arrow-datafusion that referenced this pull request Dec 15, 2025
dispanser pushed a commit to coralogix/arrow-datafusion that referenced this pull request Jan 7, 2026
dispanser pushed a commit to coralogix/arrow-datafusion that referenced this pull request Feb 19, 2026
---
[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>
dispanser pushed a commit to coralogix/arrow-datafusion that referenced this pull request Feb 26, 2026
---
[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>
avantgardnerio pushed a commit to coralogix/arrow-datafusion that referenced this pull request Mar 18, 2026
---
[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>
avantgardnerio pushed a commit to coralogix/arrow-datafusion that referenced this pull request Apr 7, 2026
---
[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>
avantgardnerio pushed a commit to coralogix/arrow-datafusion that referenced this pull request Apr 7, 2026
---
[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>
avantgardnerio added a commit to coralogix/arrow-datafusion that referenced this pull request Apr 14, 2026
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>
avantgardnerio added a commit to coralogix/arrow-datafusion that referenced this pull request Apr 29, 2026
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>
avantgardnerio added a commit to coralogix/arrow-datafusion that referenced this pull request Apr 29, 2026
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>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

physical-expr Changes to the physical-expr crates Stale PR has not had any activity for some time

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Proposal: Hook to better support CollectLeft joins in distributed execution

3 participants