Skip to content
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

Add assert on hash children partition count #5768

Merged
merged 9 commits into from
Apr 6, 2023
Merged
Show file tree
Hide file tree
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
86 changes: 45 additions & 41 deletions datafusion/core/src/physical_plan/joins/hash_join.rs
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,15 @@
//! into a set of partitions.

use ahash::RandomState;

use arrow::array::Array;
use arrow::array::{
Float32Array, Float64Array, Int16Array, Int32Array, Int64Array, Int8Array,
StringArray, TimestampNanosecondArray, UInt16Array, UInt32Array, UInt64Array,
UInt8Array,
};
use arrow::datatypes::{ArrowNativeType, DataType};
use arrow::datatypes::{Schema, SchemaRef};
use arrow::record_batch::RecordBatch;
use arrow::{
array::{
ArrayData, ArrayRef, BooleanArray, Date32Array, Date64Array, Decimal128Array,
Expand All @@ -34,27 +42,30 @@ use arrow::{
},
util::bit_util,
};
use futures::{ready, Stream, StreamExt, TryStreamExt};
use hashbrown::raw::RawTable;
use smallvec::{smallvec, SmallVec};
use std::fmt;
use std::sync::Arc;
use std::task::Poll;
use std::{any::Any, usize, vec};

use futures::{ready, Stream, StreamExt, TryStreamExt};

use arrow::array::Array;
use arrow::datatypes::{ArrowNativeType, DataType};
use arrow::datatypes::{Schema, SchemaRef};
use arrow::record_batch::RecordBatch;

use arrow::array::{
Float32Array, Float64Array, Int16Array, Int32Array, Int64Array, Int8Array,
StringArray, TimestampNanosecondArray, UInt16Array, UInt32Array, UInt64Array,
UInt8Array,
};

use datafusion_common::cast::{as_dictionary_array, as_string_array};

use hashbrown::raw::RawTable;

use crate::arrow::array::BooleanBufferBuilder;
use crate::arrow::datatypes::TimeUnit;
use crate::error::{DataFusionError, Result};
use crate::execution::{
context::TaskContext,
memory_pool::{
MemoryConsumer, SharedMemoryReservation, SharedOptionalMemoryReservation, TryGrow,
},
};
use crate::logical_expr::JoinType;
use crate::physical_plan::joins::utils::{
adjust_indices_by_join_type, apply_join_filter_to_indices, build_batch_from_indices,
get_final_indices_from_bit_map, need_produce_result_in_final, JoinSide,
};
use crate::physical_plan::{
coalesce_batches::concat_batches,
coalesce_partitions::CoalescePartitionsExec,
Expand All @@ -72,28 +83,10 @@ use crate::physical_plan::{
PhysicalExpr, RecordBatchStream, SendableRecordBatchStream, Statistics,
};

use crate::error::{DataFusionError, Result};
use crate::logical_expr::JoinType;

use crate::arrow::array::BooleanBufferBuilder;
use crate::arrow::datatypes::TimeUnit;
use crate::execution::{
context::TaskContext,
memory_pool::{
MemoryConsumer, SharedMemoryReservation, SharedOptionalMemoryReservation, TryGrow,
},
};

use super::{
utils::{OnceAsync, OnceFut},
PartitionMode,
};
use crate::physical_plan::joins::utils::{
adjust_indices_by_join_type, apply_join_filter_to_indices, build_batch_from_indices,
get_final_indices_from_bit_map, need_produce_result_in_final, JoinSide,
};
use std::fmt;
use std::task::Poll;

// Maps a `u64` hash value based on the build side ["on" values] to a list of indices with this key's value.
//
Expand Down Expand Up @@ -377,6 +370,15 @@ impl ExecutionPlan for HashJoinExec {
) -> Result<SendableRecordBatchStream> {
let on_left = self.on.iter().map(|on| on.0.clone()).collect::<Vec<_>>();
let on_right = self.on.iter().map(|on| on.1.clone()).collect::<Vec<_>>();
let left_partitions = self.left.output_partitioning().partition_count();
let right_partitions = self.right.output_partitioning().partition_count();
if self.mode == PartitionMode::Partitioned && left_partitions != right_partitions
{
return Err(DataFusionError::Internal(format!(
"Invalid HashJoinExec, partition count mismatch {left_partitions}!={right_partitions},\
consider using RepartitionExec",
)));
}

let join_metrics = BuildProbeJoinMetrics::new(partition, &self.metrics);

Expand Down Expand Up @@ -1287,7 +1289,14 @@ impl Stream for HashJoinStream {
mod tests {
use std::sync::Arc;

use super::*;
use arrow::array::{ArrayRef, Date32Array, Int32Array, UInt32Builder, UInt64Builder};
use arrow::datatypes::{DataType, Field, Schema};
use smallvec::smallvec;

use datafusion_common::ScalarValue;
use datafusion_expr::Operator;
use datafusion_physical_expr::expressions::Literal;

use crate::execution::context::SessionConfig;
use crate::physical_expr::expressions::BinaryExpr;
use crate::prelude::SessionContext;
Expand All @@ -1306,13 +1315,8 @@ mod tests {
test::exec::MockExec,
test::{build_table_i32, columns},
};
use arrow::array::{ArrayRef, Date32Array, Int32Array, UInt32Builder, UInt64Builder};
use arrow::datatypes::{DataType, Field, Schema};
use datafusion_expr::Operator;

use datafusion_common::ScalarValue;
use datafusion_physical_expr::expressions::Literal;
use smallvec::smallvec;
use super::*;

fn build_table(
a: (&str, &Vec<i32>),
Expand Down
14 changes: 13 additions & 1 deletion datafusion/core/src/physical_plan/joins/sort_merge_join.rs
Original file line number Diff line number Diff line change
Expand Up @@ -300,6 +300,15 @@ impl ExecutionPlan for SortMergeJoinExec {
partition: usize,
context: Arc<TaskContext>,
) -> Result<SendableRecordBatchStream> {
let left_partitions = self.left.output_partitioning().partition_count();
let right_partitions = self.right.output_partitioning().partition_count();
if left_partitions != right_partitions {
return Err(DataFusionError::Internal(format!(
"Invalid SortMergeJoinExec, partition count mismatch {left_partitions}!={right_partitions},\
consider using RepartitionExec",
)));
}

let (streamed, buffered, on_streamed, on_buffered) = match self.join_type {
JoinType::Inner
| JoinType::Left
Expand Down Expand Up @@ -476,6 +485,7 @@ struct StreamedBatch {
// Index of currently scanned batch from buffered data
pub buffered_batch_idx: Option<usize>,
}

impl StreamedBatch {
fn new(batch: RecordBatch, on_column: &[Column]) -> Self {
let join_arrays = join_arrays(&batch, on_column);
Expand Down Expand Up @@ -539,6 +549,7 @@ struct BufferedBatch {
/// Size estimation used for reserving / releasing memory
pub size_estimation: usize,
}

impl BufferedBatch {
fn new(batch: RecordBatch, range: Range<usize>, on_column: &[Column]) -> Self {
let join_arrays = join_arrays(&batch, on_column);
Expand Down Expand Up @@ -1153,6 +1164,7 @@ struct BufferedData {
/// current scanning offset used in join_partial()
pub scanning_offset: usize,
}

impl BufferedData {
pub fn head_batch(&self) -> &BufferedBatch {
self.batches.front().unwrap()
Expand Down Expand Up @@ -1740,7 +1752,7 @@ mod tests {
vec![
SortOptions {
descending: true,
nulls_first: false
nulls_first: false,
};
2
],
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -476,6 +476,14 @@ impl ExecutionPlan for SymmetricHashJoinExec {
partition: usize,
context: Arc<TaskContext>,
) -> Result<SendableRecordBatchStream> {
let left_partitions = self.left.output_partitioning().partition_count();
let right_partitions = self.right.output_partitioning().partition_count();
if left_partitions != right_partitions {
return Err(DataFusionError::Internal(format!(
"Invalid SymmetricHashJoinExec, partition count mismatch {left_partitions}!={right_partitions},\
consider using RepartitionExec",
)));
}
// If `filter_state` and `filter` are both present, then calculate sorted filter expressions
// for both sides, and build an expression graph if one is not already built.
let (left_sorted_filter_expr, right_sorted_filter_expr, graph) =
Expand Down