Skip to content
Merged
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
6 changes: 5 additions & 1 deletion datafusion/physical-plan/src/joins/hash_join.rs
Original file line number Diff line number Diff line change
Expand Up @@ -86,6 +86,10 @@ use datafusion_physical_expr_common::physical_expr::fmt_sql;
use futures::{ready, Stream, StreamExt, TryStreamExt};
use parking_lot::Mutex;

/// Hard-coded seed to ensure hash values from the hash join differ from `RepartitionExec`, avoiding collisions.
const HASH_JOIN_SEED: RandomState =
RandomState::with_seeds('J' as u64, 'O' as u64, 'I' as u64, 'N' as u64);
Copy link
Contributor

Choose a reason for hiding this comment

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

Love it


/// HashTable and input data for the left (build side) of a join
struct JoinLeftData {
/// The hash table with indices into `batch`
Expand Down Expand Up @@ -385,7 +389,7 @@ impl HashJoinExec {
let (join_schema, column_indices) =
build_join_schema(&left_schema, &right_schema, join_type);

let random_state = RandomState::with_seeds(0, 0, 0, 0);
let random_state = HASH_JOIN_SEED;

let join_schema = Arc::new(join_schema);

Expand Down