use std::fmt;
use std::mem::size_of;
use std::sync::Arc;
use std::task::Poll;
use std::{any::Any, usize, vec};
use crate::joins::utils::{
adjust_indices_by_join_type, apply_join_filter_to_indices, build_batch_from_indices,
calculate_join_output_ordering, combine_join_ordering_equivalence_properties,
get_final_indices_from_bit_map, need_produce_result_in_final, JoinSide,
};
use crate::DisplayAs;
use crate::{
coalesce_batches::concat_batches,
coalesce_partitions::CoalescePartitionsExec,
expressions::Column,
expressions::PhysicalSortExpr,
hash_utils::create_hashes,
joins::hash_join_utils::{JoinHashMap, JoinHashMapType},
joins::utils::{
adjust_right_output_partitioning, build_join_schema, check_join_is_valid,
combine_join_equivalence_properties, estimate_join_statistics,
partitioned_join_output_partitioning, BuildProbeJoinMetrics, ColumnIndex,
JoinFilter, JoinOn,
},
metrics::{ExecutionPlanMetricsSet, MetricsSet},
DisplayFormatType, Distribution, EquivalenceProperties, ExecutionPlan, Partitioning,
PhysicalExpr, RecordBatchStream, SendableRecordBatchStream, Statistics,
};
use super::{
utils::{OnceAsync, OnceFut},
PartitionMode,
};
use arrow::array::{
Array, ArrayRef, BooleanArray, BooleanBufferBuilder, PrimitiveArray, UInt32Array,
UInt32BufferBuilder, UInt64Array, UInt64BufferBuilder,
};
use arrow::compute::{and, take, FilterBuilder};
use arrow::datatypes::{Schema, SchemaRef};
use arrow::record_batch::RecordBatch;
use arrow::util::bit_util;
use arrow_array::cast::downcast_array;
use arrow_schema::ArrowError;
use datafusion_common::{
exec_err, internal_err, plan_err, DataFusionError, JoinType, Result,
};
use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation};
use datafusion_execution::TaskContext;
use datafusion_physical_expr::OrderingEquivalenceProperties;
use ahash::RandomState;
use arrow::compute::kernels::cmp::{eq, not_distinct};
use futures::{ready, Stream, StreamExt, TryStreamExt};
type JoinLeftData = (JoinHashMap, RecordBatch, MemoryReservation);
#[derive(Debug)]
pub struct HashJoinExec {
pub left: Arc<dyn ExecutionPlan>,
pub right: Arc<dyn ExecutionPlan>,
pub on: Vec<(Column, Column)>,
pub filter: Option<JoinFilter>,
pub join_type: JoinType,
schema: SchemaRef,
left_fut: OnceAsync<JoinLeftData>,
random_state: RandomState,
output_order: Option<Vec<PhysicalSortExpr>>,
pub mode: PartitionMode,
metrics: ExecutionPlanMetricsSet,
column_indices: Vec<ColumnIndex>,
pub null_equals_null: bool,
}
impl HashJoinExec {
pub fn try_new(
left: Arc<dyn ExecutionPlan>,
right: Arc<dyn ExecutionPlan>,
on: JoinOn,
filter: Option<JoinFilter>,
join_type: &JoinType,
partition_mode: PartitionMode,
null_equals_null: bool,
) -> Result<Self> {
let left_schema = left.schema();
let right_schema = right.schema();
if on.is_empty() {
return plan_err!("On constraints in HashJoinExec should be non-empty");
}
check_join_is_valid(&left_schema, &right_schema, &on)?;
let (schema, column_indices) =
build_join_schema(&left_schema, &right_schema, join_type);
let random_state = RandomState::with_seeds(0, 0, 0, 0);
let output_order = calculate_join_output_ordering(
left.output_ordering().unwrap_or(&[]),
right.output_ordering().unwrap_or(&[]),
*join_type,
&on,
left_schema.fields.len(),
&Self::maintains_input_order(*join_type),
Some(Self::probe_side()),
)?;
Ok(HashJoinExec {
left,
right,
on,
filter,
join_type: *join_type,
schema: Arc::new(schema),
left_fut: Default::default(),
random_state,
mode: partition_mode,
metrics: ExecutionPlanMetricsSet::new(),
column_indices,
null_equals_null,
output_order,
})
}
pub fn left(&self) -> &Arc<dyn ExecutionPlan> {
&self.left
}
pub fn right(&self) -> &Arc<dyn ExecutionPlan> {
&self.right
}
pub fn on(&self) -> &[(Column, Column)] {
&self.on
}
pub fn filter(&self) -> Option<&JoinFilter> {
self.filter.as_ref()
}
pub fn join_type(&self) -> &JoinType {
&self.join_type
}
pub fn partition_mode(&self) -> &PartitionMode {
&self.mode
}
pub fn null_equals_null(&self) -> bool {
self.null_equals_null
}
fn maintains_input_order(join_type: JoinType) -> Vec<bool> {
vec![
false,
matches!(
join_type,
JoinType::Inner | JoinType::RightAnti | JoinType::RightSemi
),
]
}
pub fn probe_side() -> JoinSide {
JoinSide::Right
}
}
impl DisplayAs for HashJoinExec {
fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result {
match t {
DisplayFormatType::Default | DisplayFormatType::Verbose => {
let display_filter = self.filter.as_ref().map_or_else(
|| "".to_string(),
|f| format!(", filter={}", f.expression()),
);
let on = self
.on
.iter()
.map(|(c1, c2)| format!("({}, {})", c1, c2))
.collect::<Vec<String>>()
.join(", ");
write!(
f,
"HashJoinExec: mode={:?}, join_type={:?}, on=[{}]{}",
self.mode, self.join_type, on, display_filter
)
}
}
}
}
impl ExecutionPlan for HashJoinExec {
fn as_any(&self) -> &dyn Any {
self
}
fn schema(&self) -> SchemaRef {
self.schema.clone()
}
fn required_input_distribution(&self) -> Vec<Distribution> {
match self.mode {
PartitionMode::CollectLeft => vec![
Distribution::SinglePartition,
Distribution::UnspecifiedDistribution,
],
PartitionMode::Partitioned => {
let (left_expr, right_expr) = self
.on
.iter()
.map(|(l, r)| {
(
Arc::new(l.clone()) as Arc<dyn PhysicalExpr>,
Arc::new(r.clone()) as Arc<dyn PhysicalExpr>,
)
})
.unzip();
vec![
Distribution::HashPartitioned(left_expr),
Distribution::HashPartitioned(right_expr),
]
}
PartitionMode::Auto => vec![
Distribution::UnspecifiedDistribution,
Distribution::UnspecifiedDistribution,
],
}
}
fn unbounded_output(&self, children: &[bool]) -> Result<bool> {
let (left, right) = (children[0], children[1]);
let breaking = left
|| (right
&& matches!(
self.join_type,
JoinType::Left
| JoinType::Full
| JoinType::LeftAnti
| JoinType::LeftSemi
));
if breaking {
plan_err!(
"Join Error: The join with cannot be executed with unbounded inputs. {}",
if left && right {
"Currently, we do not support unbounded inputs on both sides."
} else {
"Please consider a different type of join or sources."
}
)
} else {
Ok(left || right)
}
}
fn output_partitioning(&self) -> Partitioning {
let left_columns_len = self.left.schema().fields.len();
match self.mode {
PartitionMode::CollectLeft => match self.join_type {
JoinType::Inner | JoinType::Right => adjust_right_output_partitioning(
self.right.output_partitioning(),
left_columns_len,
),
JoinType::RightSemi | JoinType::RightAnti => {
self.right.output_partitioning()
}
JoinType::Left
| JoinType::LeftSemi
| JoinType::LeftAnti
| JoinType::Full => Partitioning::UnknownPartitioning(
self.right.output_partitioning().partition_count(),
),
},
PartitionMode::Partitioned => partitioned_join_output_partitioning(
self.join_type,
self.left.output_partitioning(),
self.right.output_partitioning(),
left_columns_len,
),
PartitionMode::Auto => Partitioning::UnknownPartitioning(
self.right.output_partitioning().partition_count(),
),
}
}
fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> {
self.output_order.as_deref()
}
fn maintains_input_order(&self) -> Vec<bool> {
Self::maintains_input_order(self.join_type)
}
fn equivalence_properties(&self) -> EquivalenceProperties {
let left_columns_len = self.left.schema().fields.len();
combine_join_equivalence_properties(
self.join_type,
self.left.equivalence_properties(),
self.right.equivalence_properties(),
left_columns_len,
self.on(),
self.schema(),
)
}
fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties {
combine_join_ordering_equivalence_properties(
&self.join_type,
&self.left,
&self.right,
self.schema(),
&self.maintains_input_order(),
Some(Self::probe_side()),
self.equivalence_properties(),
)
.unwrap()
}
fn children(&self) -> Vec<Arc<dyn ExecutionPlan>> {
vec![self.left.clone(), self.right.clone()]
}
fn with_new_children(
self: Arc<Self>,
children: Vec<Arc<dyn ExecutionPlan>>,
) -> Result<Arc<dyn ExecutionPlan>> {
Ok(Arc::new(HashJoinExec::try_new(
children[0].clone(),
children[1].clone(),
self.on.clone(),
self.filter.clone(),
&self.join_type,
self.mode,
self.null_equals_null,
)?))
}
fn execute(
&self,
partition: usize,
context: Arc<TaskContext>,
) -> 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 internal_err!(
"Invalid HashJoinExec, partition count mismatch {left_partitions}!={right_partitions},\
consider using RepartitionExec"
);
}
let join_metrics = BuildProbeJoinMetrics::new(partition, &self.metrics);
let left_fut = match self.mode {
PartitionMode::CollectLeft => self.left_fut.once(|| {
let reservation =
MemoryConsumer::new("HashJoinInput").register(context.memory_pool());
collect_left_input(
None,
self.random_state.clone(),
self.left.clone(),
on_left.clone(),
context.clone(),
join_metrics.clone(),
reservation,
)
}),
PartitionMode::Partitioned => {
let reservation =
MemoryConsumer::new(format!("HashJoinInput[{partition}]"))
.register(context.memory_pool());
OnceFut::new(collect_left_input(
Some(partition),
self.random_state.clone(),
self.left.clone(),
on_left.clone(),
context.clone(),
join_metrics.clone(),
reservation,
))
}
PartitionMode::Auto => {
return plan_err!(
"Invalid HashJoinExec, unsupported PartitionMode {:?} in execute()",
PartitionMode::Auto
);
}
};
let reservation = MemoryConsumer::new(format!("HashJoinStream[{partition}]"))
.register(context.memory_pool());
let right_stream = self.right.execute(partition, context)?;
Ok(Box::pin(HashJoinStream {
schema: self.schema(),
on_left,
on_right,
filter: self.filter.clone(),
join_type: self.join_type,
left_fut,
visited_left_side: None,
right: right_stream,
column_indices: self.column_indices.clone(),
random_state: self.random_state.clone(),
join_metrics,
null_equals_null: self.null_equals_null,
is_exhausted: false,
reservation,
}))
}
fn metrics(&self) -> Option<MetricsSet> {
Some(self.metrics.clone_inner())
}
fn statistics(&self) -> Statistics {
estimate_join_statistics(
self.left.clone(),
self.right.clone(),
self.on.clone(),
&self.join_type,
)
}
}
async fn collect_left_input(
partition: Option<usize>,
random_state: RandomState,
left: Arc<dyn ExecutionPlan>,
on_left: Vec<Column>,
context: Arc<TaskContext>,
metrics: BuildProbeJoinMetrics,
reservation: MemoryReservation,
) -> Result<JoinLeftData> {
let schema = left.schema();
let (left_input, left_input_partition) = if let Some(partition) = partition {
(left, partition)
} else {
let merge = {
if left.output_partitioning().partition_count() != 1 {
Arc::new(CoalescePartitionsExec::new(left))
} else {
left
}
};
(merge, 0)
};
let stream = left_input.execute(left_input_partition, context.clone())?;
let initial = (Vec::new(), 0, metrics, reservation);
let (batches, num_rows, metrics, mut reservation) = stream
.try_fold(initial, |mut acc, batch| async {
let batch_size = batch.get_array_memory_size();
acc.3.try_grow(batch_size)?;
acc.2.build_mem_used.add(batch_size);
acc.2.build_input_batches.add(1);
acc.2.build_input_rows.add(batch.num_rows());
acc.1 += batch.num_rows();
acc.0.push(batch);
Ok(acc)
})
.await?;
let estimated_buckets = (num_rows.checked_mul(8).ok_or_else(|| {
DataFusionError::Execution(
"usize overflow while estimating number of hasmap buckets".to_string(),
)
})? / 7)
.next_power_of_two();
let estimated_hastable_size =
16 * estimated_buckets + estimated_buckets + size_of::<JoinHashMap>();
reservation.try_grow(estimated_hastable_size)?;
metrics.build_mem_used.add(estimated_hastable_size);
let mut hashmap = JoinHashMap::with_capacity(num_rows);
let mut hashes_buffer = Vec::new();
let mut offset = 0;
for batch in batches.iter() {
hashes_buffer.clear();
hashes_buffer.resize(batch.num_rows(), 0);
update_hash(
&on_left,
batch,
&mut hashmap,
offset,
&random_state,
&mut hashes_buffer,
0,
)?;
offset += batch.num_rows();
}
let single_batch = concat_batches(&schema, &batches, num_rows)?;
Ok((hashmap, single_batch, reservation))
}
pub fn update_hash<T>(
on: &[Column],
batch: &RecordBatch,
hash_map: &mut T,
offset: usize,
random_state: &RandomState,
hashes_buffer: &mut Vec<u64>,
deleted_offset: usize,
) -> Result<()>
where
T: JoinHashMapType,
{
let keys_values = on
.iter()
.map(|c| Ok(c.evaluate(batch)?.into_array(batch.num_rows())))
.collect::<Result<Vec<_>>>()?;
let hash_values = create_hashes(&keys_values, random_state, hashes_buffer)?;
hash_map.extend_zero(batch.num_rows());
let (mut_map, mut_list) = hash_map.get_mut();
for (row, hash_value) in hash_values.iter().enumerate() {
let item = mut_map.get_mut(*hash_value, |(hash, _)| *hash_value == *hash);
if let Some((_, index)) = item {
let prev_index = *index;
*index = (row + offset + 1) as u64;
mut_list[row + offset - deleted_offset] = prev_index;
} else {
mut_map.insert(
*hash_value,
(*hash_value, (row + offset + 1) as u64),
|(hash, _)| *hash,
);
}
}
Ok(())
}
struct HashJoinStream {
schema: Arc<Schema>,
on_left: Vec<Column>,
on_right: Vec<Column>,
filter: Option<JoinFilter>,
join_type: JoinType,
left_fut: OnceFut<JoinLeftData>,
visited_left_side: Option<BooleanBufferBuilder>,
right: SendableRecordBatchStream,
random_state: RandomState,
is_exhausted: bool,
join_metrics: BuildProbeJoinMetrics,
column_indices: Vec<ColumnIndex>,
null_equals_null: bool,
reservation: MemoryReservation,
}
impl RecordBatchStream for HashJoinStream {
fn schema(&self) -> SchemaRef {
self.schema.clone()
}
}
#[allow(clippy::too_many_arguments)]
pub fn build_equal_condition_join_indices<T: JoinHashMapType>(
build_hashmap: &T,
build_input_buffer: &RecordBatch,
probe_batch: &RecordBatch,
build_on: &[Column],
probe_on: &[Column],
random_state: &RandomState,
null_equals_null: bool,
hashes_buffer: &mut Vec<u64>,
filter: Option<&JoinFilter>,
build_side: JoinSide,
deleted_offset: Option<usize>,
) -> Result<(UInt64Array, UInt32Array)> {
let keys_values = probe_on
.iter()
.map(|c| Ok(c.evaluate(probe_batch)?.into_array(probe_batch.num_rows())))
.collect::<Result<Vec<_>>>()?;
let build_join_values = build_on
.iter()
.map(|c| {
Ok(c.evaluate(build_input_buffer)?
.into_array(build_input_buffer.num_rows()))
})
.collect::<Result<Vec<_>>>()?;
hashes_buffer.clear();
hashes_buffer.resize(probe_batch.num_rows(), 0);
let hash_values = create_hashes(&keys_values, random_state, hashes_buffer)?;
let mut build_indices = UInt64BufferBuilder::new(0);
let mut probe_indices = UInt32BufferBuilder::new(0);
let hash_map = build_hashmap.get_map();
let next_chain = build_hashmap.get_list();
for (row, hash_value) in hash_values.iter().enumerate().rev() {
if let Some((_, index)) =
hash_map.get(*hash_value, |(hash, _)| *hash_value == *hash)
{
let mut i = *index - 1;
loop {
let build_row_value = if let Some(offset) = deleted_offset {
if i < offset as u64 {
break;
}
i - offset as u64
} else {
i
};
build_indices.append(build_row_value);
probe_indices.append(row as u32);
let next = next_chain[build_row_value as usize];
if next == 0 {
break;
}
i = next - 1;
}
}
}
build_indices.as_slice_mut().reverse();
probe_indices.as_slice_mut().reverse();
let left: UInt64Array = PrimitiveArray::new(build_indices.finish().into(), None);
let right: UInt32Array = PrimitiveArray::new(probe_indices.finish().into(), None);
let (left, right) = if let Some(filter) = filter {
apply_join_filter_to_indices(
build_input_buffer,
probe_batch,
left,
right,
filter,
build_side,
)?
} else {
(left, right)
};
equal_rows_arr(
&left,
&right,
&build_join_values,
&keys_values,
null_equals_null,
)
}
fn eq_dyn_null(
left: &dyn Array,
right: &dyn Array,
null_equals_null: bool,
) -> Result<BooleanArray, ArrowError> {
match (left.data_type(), right.data_type()) {
_ if null_equals_null => not_distinct(&left, &right),
_ => eq(&left, &right),
}
}
pub fn equal_rows_arr(
indices_left: &UInt64Array,
indices_right: &UInt32Array,
left_arrays: &[ArrayRef],
right_arrays: &[ArrayRef],
null_equals_null: bool,
) -> Result<(UInt64Array, UInt32Array)> {
let mut iter = left_arrays.iter().zip(right_arrays.iter());
let (first_left, first_right) = iter.next().ok_or_else(|| {
DataFusionError::Internal(
"At least one array should be provided for both left and right".to_string(),
)
})?;
let arr_left = take(first_left.as_ref(), indices_left, None)?;
let arr_right = take(first_right.as_ref(), indices_right, None)?;
let mut equal: BooleanArray = eq_dyn_null(&arr_left, &arr_right, null_equals_null)?;
equal = iter
.map(|(left, right)| {
let arr_left = take(left.as_ref(), indices_left, None)?;
let arr_right = take(right.as_ref(), indices_right, None)?;
eq_dyn_null(arr_left.as_ref(), arr_right.as_ref(), null_equals_null)
})
.try_fold(equal, |acc, equal2| and(&acc, &equal2?))?;
let filter_builder = FilterBuilder::new(&equal).optimize().build();
let left_filtered = filter_builder.filter(indices_left)?;
let right_filtered = filter_builder.filter(indices_right)?;
Ok((
downcast_array(left_filtered.as_ref()),
downcast_array(right_filtered.as_ref()),
))
}
impl HashJoinStream {
fn poll_next_impl(
&mut self,
cx: &mut std::task::Context<'_>,
) -> Poll<Option<Result<RecordBatch>>> {
let build_timer = self.join_metrics.build_time.timer();
let left_data = match ready!(self.left_fut.get(cx)) {
Ok(left_data) => left_data,
Err(e) => return Poll::Ready(Some(Err(e))),
};
build_timer.done();
if self.visited_left_side.is_none()
&& need_produce_result_in_final(self.join_type)
{
let visited_bitmap_size = bit_util::ceil(left_data.1.num_rows(), 8);
self.reservation.try_grow(visited_bitmap_size)?;
self.join_metrics.build_mem_used.add(visited_bitmap_size);
}
let visited_left_side = self.visited_left_side.get_or_insert_with(|| {
let num_rows = left_data.1.num_rows();
if need_produce_result_in_final(self.join_type) {
let mut buffer = BooleanBufferBuilder::new(num_rows);
buffer.append_n(num_rows, false);
buffer
} else {
BooleanBufferBuilder::new(0)
}
});
let mut hashes_buffer = vec![];
self.right
.poll_next_unpin(cx)
.map(|maybe_batch| match maybe_batch {
Some(Ok(batch)) => {
self.join_metrics.input_batches.add(1);
self.join_metrics.input_rows.add(batch.num_rows());
let timer = self.join_metrics.join_time.timer();
let left_right_indices = build_equal_condition_join_indices(
&left_data.0,
&left_data.1,
&batch,
&self.on_left,
&self.on_right,
&self.random_state,
self.null_equals_null,
&mut hashes_buffer,
self.filter.as_ref(),
JoinSide::Left,
None,
);
let result = match left_right_indices {
Ok((left_side, right_side)) => {
if need_produce_result_in_final(self.join_type) {
left_side.iter().flatten().for_each(|x| {
visited_left_side.set_bit(x as usize, true);
});
}
let (left_side, right_side) = adjust_indices_by_join_type(
left_side,
right_side,
batch.num_rows(),
self.join_type,
);
let result = build_batch_from_indices(
&self.schema,
&left_data.1,
&batch,
&left_side,
&right_side,
&self.column_indices,
JoinSide::Left,
);
self.join_metrics.output_batches.add(1);
self.join_metrics.output_rows.add(batch.num_rows());
Some(result)
}
Err(err) => Some(exec_err!(
"Fail to build join indices in HashJoinExec, error:{err}"
)),
};
timer.done();
result
}
None => {
let timer = self.join_metrics.join_time.timer();
if need_produce_result_in_final(self.join_type) && !self.is_exhausted
{
let (left_side, right_side) = get_final_indices_from_bit_map(
visited_left_side,
self.join_type,
);
let empty_right_batch =
RecordBatch::new_empty(self.right.schema());
let result = build_batch_from_indices(
&self.schema,
&left_data.1,
&empty_right_batch,
&left_side,
&right_side,
&self.column_indices,
JoinSide::Left,
);
if let Ok(ref batch) = result {
self.join_metrics.input_batches.add(1);
self.join_metrics.input_rows.add(batch.num_rows());
self.join_metrics.output_batches.add(1);
self.join_metrics.output_rows.add(batch.num_rows());
}
timer.done();
self.is_exhausted = true;
Some(result)
} else {
None
}
}
Some(err) => Some(err),
})
}
}
impl Stream for HashJoinStream {
type Item = Result<RecordBatch>;
fn poll_next(
mut self: std::pin::Pin<&mut Self>,
cx: &mut std::task::Context<'_>,
) -> std::task::Poll<Option<Self::Item>> {
self.poll_next_impl(cx)
}
}
#[cfg(test)]
mod tests {
use std::sync::Arc;
use arrow::array::{ArrayRef, Date32Array, Int32Array, UInt32Builder, UInt64Builder};
use arrow::datatypes::{DataType, Field, Schema};
use datafusion_common::{assert_batches_sorted_eq, assert_contains, ScalarValue};
use datafusion_expr::Operator;
use datafusion_physical_expr::expressions::Literal;
use hashbrown::raw::RawTable;
use crate::{
common,
expressions::Column,
hash_utils::create_hashes,
joins::{hash_join::build_equal_condition_join_indices, utils::JoinSide},
memory::MemoryExec,
repartition::RepartitionExec,
test::build_table_i32,
test::exec::MockExec,
};
use datafusion_execution::config::SessionConfig;
use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv};
use datafusion_physical_expr::expressions::BinaryExpr;
use super::*;
fn build_table(
a: (&str, &Vec<i32>),
b: (&str, &Vec<i32>),
c: (&str, &Vec<i32>),
) -> Arc<dyn ExecutionPlan> {
let batch = build_table_i32(a, b, c);
let schema = batch.schema();
Arc::new(MemoryExec::try_new(&[vec![batch]], schema, None).unwrap())
}
fn join(
left: Arc<dyn ExecutionPlan>,
right: Arc<dyn ExecutionPlan>,
on: JoinOn,
join_type: &JoinType,
null_equals_null: bool,
) -> Result<HashJoinExec> {
HashJoinExec::try_new(
left,
right,
on,
None,
join_type,
PartitionMode::CollectLeft,
null_equals_null,
)
}
fn join_with_filter(
left: Arc<dyn ExecutionPlan>,
right: Arc<dyn ExecutionPlan>,
on: JoinOn,
filter: JoinFilter,
join_type: &JoinType,
null_equals_null: bool,
) -> Result<HashJoinExec> {
HashJoinExec::try_new(
left,
right,
on,
Some(filter),
join_type,
PartitionMode::CollectLeft,
null_equals_null,
)
}
async fn join_collect(
left: Arc<dyn ExecutionPlan>,
right: Arc<dyn ExecutionPlan>,
on: JoinOn,
join_type: &JoinType,
null_equals_null: bool,
context: Arc<TaskContext>,
) -> Result<(Vec<String>, Vec<RecordBatch>)> {
let join = join(left, right, on, join_type, null_equals_null)?;
let columns_header = columns(&join.schema());
let stream = join.execute(0, context)?;
let batches = common::collect(stream).await?;
Ok((columns_header, batches))
}
async fn partitioned_join_collect(
left: Arc<dyn ExecutionPlan>,
right: Arc<dyn ExecutionPlan>,
on: JoinOn,
join_type: &JoinType,
null_equals_null: bool,
context: Arc<TaskContext>,
) -> Result<(Vec<String>, Vec<RecordBatch>)> {
let partition_count = 4;
let (left_expr, right_expr) = on
.iter()
.map(|(l, r)| {
(
Arc::new(l.clone()) as Arc<dyn PhysicalExpr>,
Arc::new(r.clone()) as Arc<dyn PhysicalExpr>,
)
})
.unzip();
let join = HashJoinExec::try_new(
Arc::new(RepartitionExec::try_new(
left,
Partitioning::Hash(left_expr, partition_count),
)?),
Arc::new(RepartitionExec::try_new(
right,
Partitioning::Hash(right_expr, partition_count),
)?),
on,
None,
join_type,
PartitionMode::Partitioned,
null_equals_null,
)?;
let columns = columns(&join.schema());
let mut batches = vec![];
for i in 0..partition_count {
let stream = join.execute(i, context.clone())?;
let more_batches = common::collect(stream).await?;
batches.extend(
more_batches
.into_iter()
.filter(|b| b.num_rows() > 0)
.collect::<Vec<_>>(),
);
}
Ok((columns, batches))
}
#[tokio::test]
async fn join_inner_one() -> Result<()> {
let task_ctx = Arc::new(TaskContext::default());
let left = build_table(
("a1", &vec![1, 2, 3]),
("b1", &vec![4, 5, 5]), ("c1", &vec![7, 8, 9]),
);
let right = build_table(
("a2", &vec![10, 20, 30]),
("b1", &vec![4, 5, 6]),
("c2", &vec![70, 80, 90]),
);
let on = vec![(
Column::new_with_schema("b1", &left.schema())?,
Column::new_with_schema("b1", &right.schema())?,
)];
let (columns, batches) = join_collect(
left.clone(),
right.clone(),
on.clone(),
&JoinType::Inner,
false,
task_ctx,
)
.await?;
assert_eq!(columns, vec!["a1", "b1", "c1", "a2", "b1", "c2"]);
let expected = [
"+----+----+----+----+----+----+",
"| a1 | b1 | c1 | a2 | b1 | c2 |",
"+----+----+----+----+----+----+",
"| 1 | 4 | 7 | 10 | 4 | 70 |",
"| 2 | 5 | 8 | 20 | 5 | 80 |",
"| 3 | 5 | 9 | 20 | 5 | 80 |",
"+----+----+----+----+----+----+",
];
assert_batches_sorted_eq!(expected, &batches);
Ok(())
}
#[tokio::test]
async fn partitioned_join_inner_one() -> Result<()> {
let task_ctx = Arc::new(TaskContext::default());
let left = build_table(
("a1", &vec![1, 2, 3]),
("b1", &vec![4, 5, 5]), ("c1", &vec![7, 8, 9]),
);
let right = build_table(
("a2", &vec![10, 20, 30]),
("b1", &vec![4, 5, 6]),
("c2", &vec![70, 80, 90]),
);
let on = vec![(
Column::new_with_schema("b1", &left.schema())?,
Column::new_with_schema("b1", &right.schema())?,
)];
let (columns, batches) = partitioned_join_collect(
left.clone(),
right.clone(),
on.clone(),
&JoinType::Inner,
false,
task_ctx,
)
.await?;
assert_eq!(columns, vec!["a1", "b1", "c1", "a2", "b1", "c2"]);
let expected = [
"+----+----+----+----+----+----+",
"| a1 | b1 | c1 | a2 | b1 | c2 |",
"+----+----+----+----+----+----+",
"| 1 | 4 | 7 | 10 | 4 | 70 |",
"| 2 | 5 | 8 | 20 | 5 | 80 |",
"| 3 | 5 | 9 | 20 | 5 | 80 |",
"+----+----+----+----+----+----+",
];
assert_batches_sorted_eq!(expected, &batches);
Ok(())
}
#[tokio::test]
async fn join_inner_one_no_shared_column_names() -> Result<()> {
let task_ctx = Arc::new(TaskContext::default());
let left = build_table(
("a1", &vec![1, 2, 3]),
("b1", &vec![4, 5, 5]), ("c1", &vec![7, 8, 9]),
);
let right = build_table(
("a2", &vec![10, 20, 30]),
("b2", &vec![4, 5, 6]),
("c2", &vec![70, 80, 90]),
);
let on = vec![(
Column::new_with_schema("b1", &left.schema())?,
Column::new_with_schema("b2", &right.schema())?,
)];
let (columns, batches) =
join_collect(left, right, on, &JoinType::Inner, false, task_ctx).await?;
assert_eq!(columns, vec!["a1", "b1", "c1", "a2", "b2", "c2"]);
let expected = [
"+----+----+----+----+----+----+",
"| a1 | b1 | c1 | a2 | b2 | c2 |",
"+----+----+----+----+----+----+",
"| 1 | 4 | 7 | 10 | 4 | 70 |",
"| 2 | 5 | 8 | 20 | 5 | 80 |",
"| 3 | 5 | 9 | 20 | 5 | 80 |",
"+----+----+----+----+----+----+",
];
assert_batches_sorted_eq!(expected, &batches);
Ok(())
}
#[tokio::test]
async fn join_inner_two() -> Result<()> {
let task_ctx = Arc::new(TaskContext::default());
let left = build_table(
("a1", &vec![1, 2, 2]),
("b2", &vec![1, 2, 2]),
("c1", &vec![7, 8, 9]),
);
let right = build_table(
("a1", &vec![1, 2, 3]),
("b2", &vec![1, 2, 2]),
("c2", &vec![70, 80, 90]),
);
let on = vec![
(
Column::new_with_schema("a1", &left.schema())?,
Column::new_with_schema("a1", &right.schema())?,
),
(
Column::new_with_schema("b2", &left.schema())?,
Column::new_with_schema("b2", &right.schema())?,
),
];
let (columns, batches) =
join_collect(left, right, on, &JoinType::Inner, false, task_ctx).await?;
assert_eq!(columns, vec!["a1", "b2", "c1", "a1", "b2", "c2"]);
assert_eq!(batches.len(), 1);
let expected = [
"+----+----+----+----+----+----+",
"| a1 | b2 | c1 | a1 | b2 | c2 |",
"+----+----+----+----+----+----+",
"| 1 | 1 | 7 | 1 | 1 | 70 |",
"| 2 | 2 | 8 | 2 | 2 | 80 |",
"| 2 | 2 | 9 | 2 | 2 | 80 |",
"+----+----+----+----+----+----+",
];
assert_batches_sorted_eq!(expected, &batches);
Ok(())
}
#[tokio::test]
async fn join_inner_one_two_parts_left() -> Result<()> {
let task_ctx = Arc::new(TaskContext::default());
let batch1 = build_table_i32(
("a1", &vec![1, 2]),
("b2", &vec![1, 2]),
("c1", &vec![7, 8]),
);
let batch2 =
build_table_i32(("a1", &vec![2]), ("b2", &vec![2]), ("c1", &vec![9]));
let schema = batch1.schema();
let left = Arc::new(
MemoryExec::try_new(&[vec![batch1], vec![batch2]], schema, None).unwrap(),
);
let right = build_table(
("a1", &vec![1, 2, 3]),
("b2", &vec![1, 2, 2]),
("c2", &vec![70, 80, 90]),
);
let on = vec![
(
Column::new_with_schema("a1", &left.schema())?,
Column::new_with_schema("a1", &right.schema())?,
),
(
Column::new_with_schema("b2", &left.schema())?,
Column::new_with_schema("b2", &right.schema())?,
),
];
let (columns, batches) =
join_collect(left, right, on, &JoinType::Inner, false, task_ctx).await?;
assert_eq!(columns, vec!["a1", "b2", "c1", "a1", "b2", "c2"]);
assert_eq!(batches.len(), 1);
let expected = [
"+----+----+----+----+----+----+",
"| a1 | b2 | c1 | a1 | b2 | c2 |",
"+----+----+----+----+----+----+",
"| 1 | 1 | 7 | 1 | 1 | 70 |",
"| 2 | 2 | 8 | 2 | 2 | 80 |",
"| 2 | 2 | 9 | 2 | 2 | 80 |",
"+----+----+----+----+----+----+",
];
assert_batches_sorted_eq!(expected, &batches);
Ok(())
}
#[tokio::test]
async fn join_inner_one_two_parts_right() -> Result<()> {
let task_ctx = Arc::new(TaskContext::default());
let left = build_table(
("a1", &vec![1, 2, 3]),
("b1", &vec![4, 5, 5]), ("c1", &vec![7, 8, 9]),
);
let batch1 = build_table_i32(
("a2", &vec![10, 20]),
("b1", &vec![4, 6]),
("c2", &vec![70, 80]),
);
let batch2 =
build_table_i32(("a2", &vec![30]), ("b1", &vec![5]), ("c2", &vec![90]));
let schema = batch1.schema();
let right = Arc::new(
MemoryExec::try_new(&[vec![batch1], vec![batch2]], schema, None).unwrap(),
);
let on = vec![(
Column::new_with_schema("b1", &left.schema())?,
Column::new_with_schema("b1", &right.schema())?,
)];
let join = join(left, right, on, &JoinType::Inner, false)?;
let columns = columns(&join.schema());
assert_eq!(columns, vec!["a1", "b1", "c1", "a2", "b1", "c2"]);
let stream = join.execute(0, task_ctx.clone())?;
let batches = common::collect(stream).await?;
assert_eq!(batches.len(), 1);
let expected = [
"+----+----+----+----+----+----+",
"| a1 | b1 | c1 | a2 | b1 | c2 |",
"+----+----+----+----+----+----+",
"| 1 | 4 | 7 | 10 | 4 | 70 |",
"+----+----+----+----+----+----+",
];
assert_batches_sorted_eq!(expected, &batches);
let stream = join.execute(1, task_ctx.clone())?;
let batches = common::collect(stream).await?;
assert_eq!(batches.len(), 1);
let expected = [
"+----+----+----+----+----+----+",
"| a1 | b1 | c1 | a2 | b1 | c2 |",
"+----+----+----+----+----+----+",
"| 2 | 5 | 8 | 30 | 5 | 90 |",
"| 3 | 5 | 9 | 30 | 5 | 90 |",
"+----+----+----+----+----+----+",
];
assert_batches_sorted_eq!(expected, &batches);
Ok(())
}
fn build_table_two_batches(
a: (&str, &Vec<i32>),
b: (&str, &Vec<i32>),
c: (&str, &Vec<i32>),
) -> Arc<dyn ExecutionPlan> {
let batch = build_table_i32(a, b, c);
let schema = batch.schema();
Arc::new(
MemoryExec::try_new(&[vec![batch.clone(), batch]], schema, None).unwrap(),
)
}
#[tokio::test]
async fn join_left_multi_batch() {
let task_ctx = Arc::new(TaskContext::default());
let left = build_table(
("a1", &vec![1, 2, 3]),
("b1", &vec![4, 5, 7]), ("c1", &vec![7, 8, 9]),
);
let right = build_table_two_batches(
("a2", &vec![10, 20, 30]),
("b1", &vec![4, 5, 6]),
("c2", &vec![70, 80, 90]),
);
let on = vec![(
Column::new_with_schema("b1", &left.schema()).unwrap(),
Column::new_with_schema("b1", &right.schema()).unwrap(),
)];
let join = join(left, right, on, &JoinType::Left, false).unwrap();
let columns = columns(&join.schema());
assert_eq!(columns, vec!["a1", "b1", "c1", "a2", "b1", "c2"]);
let stream = join.execute(0, task_ctx).unwrap();
let batches = common::collect(stream).await.unwrap();
let expected = [
"+----+----+----+----+----+----+",
"| a1 | b1 | c1 | a2 | b1 | c2 |",
"+----+----+----+----+----+----+",
"| 1 | 4 | 7 | 10 | 4 | 70 |",
"| 1 | 4 | 7 | 10 | 4 | 70 |",
"| 2 | 5 | 8 | 20 | 5 | 80 |",
"| 2 | 5 | 8 | 20 | 5 | 80 |",
"| 3 | 7 | 9 | | | |",
"+----+----+----+----+----+----+",
];
assert_batches_sorted_eq!(expected, &batches);
}
#[tokio::test]
async fn join_full_multi_batch() {
let task_ctx = Arc::new(TaskContext::default());
let left = build_table(
("a1", &vec![1, 2, 3]),
("b1", &vec![4, 5, 7]), ("c1", &vec![7, 8, 9]),
);
let right = build_table_two_batches(
("a2", &vec![10, 20, 30]),
("b2", &vec![4, 5, 6]),
("c2", &vec![70, 80, 90]),
);
let on = vec![(
Column::new_with_schema("b1", &left.schema()).unwrap(),
Column::new_with_schema("b2", &right.schema()).unwrap(),
)];
let join = join(left, right, on, &JoinType::Full, false).unwrap();
let columns = columns(&join.schema());
assert_eq!(columns, vec!["a1", "b1", "c1", "a2", "b2", "c2"]);
let stream = join.execute(0, task_ctx).unwrap();
let batches = common::collect(stream).await.unwrap();
let expected = [
"+----+----+----+----+----+----+",
"| a1 | b1 | c1 | a2 | b2 | c2 |",
"+----+----+----+----+----+----+",
"| | | | 30 | 6 | 90 |",
"| | | | 30 | 6 | 90 |",
"| 1 | 4 | 7 | 10 | 4 | 70 |",
"| 1 | 4 | 7 | 10 | 4 | 70 |",
"| 2 | 5 | 8 | 20 | 5 | 80 |",
"| 2 | 5 | 8 | 20 | 5 | 80 |",
"| 3 | 7 | 9 | | | |",
"+----+----+----+----+----+----+",
];
assert_batches_sorted_eq!(expected, &batches);
}
#[tokio::test]
async fn join_left_empty_right() {
let task_ctx = Arc::new(TaskContext::default());
let left = build_table(
("a1", &vec![1, 2, 3]),
("b1", &vec![4, 5, 7]),
("c1", &vec![7, 8, 9]),
);
let right = build_table_i32(("a2", &vec![]), ("b1", &vec![]), ("c2", &vec![]));
let on = vec![(
Column::new_with_schema("b1", &left.schema()).unwrap(),
Column::new_with_schema("b1", &right.schema()).unwrap(),
)];
let schema = right.schema();
let right = Arc::new(MemoryExec::try_new(&[vec![right]], schema, None).unwrap());
let join = join(left, right, on, &JoinType::Left, false).unwrap();
let columns = columns(&join.schema());
assert_eq!(columns, vec!["a1", "b1", "c1", "a2", "b1", "c2"]);
let stream = join.execute(0, task_ctx).unwrap();
let batches = common::collect(stream).await.unwrap();
let expected = [
"+----+----+----+----+----+----+",
"| a1 | b1 | c1 | a2 | b1 | c2 |",
"+----+----+----+----+----+----+",
"| 1 | 4 | 7 | | | |",
"| 2 | 5 | 8 | | | |",
"| 3 | 7 | 9 | | | |",
"+----+----+----+----+----+----+",
];
assert_batches_sorted_eq!(expected, &batches);
}
#[tokio::test]
async fn join_full_empty_right() {
let task_ctx = Arc::new(TaskContext::default());
let left = build_table(
("a1", &vec![1, 2, 3]),
("b1", &vec![4, 5, 7]),
("c1", &vec![7, 8, 9]),
);
let right = build_table_i32(("a2", &vec![]), ("b2", &vec![]), ("c2", &vec![]));
let on = vec![(
Column::new_with_schema("b1", &left.schema()).unwrap(),
Column::new_with_schema("b2", &right.schema()).unwrap(),
)];
let schema = right.schema();
let right = Arc::new(MemoryExec::try_new(&[vec![right]], schema, None).unwrap());
let join = join(left, right, on, &JoinType::Full, false).unwrap();
let columns = columns(&join.schema());
assert_eq!(columns, vec!["a1", "b1", "c1", "a2", "b2", "c2"]);
let stream = join.execute(0, task_ctx).unwrap();
let batches = common::collect(stream).await.unwrap();
let expected = [
"+----+----+----+----+----+----+",
"| a1 | b1 | c1 | a2 | b2 | c2 |",
"+----+----+----+----+----+----+",
"| 1 | 4 | 7 | | | |",
"| 2 | 5 | 8 | | | |",
"| 3 | 7 | 9 | | | |",
"+----+----+----+----+----+----+",
];
assert_batches_sorted_eq!(expected, &batches);
}
#[tokio::test]
async fn join_left_one() -> Result<()> {
let task_ctx = Arc::new(TaskContext::default());
let left = build_table(
("a1", &vec![1, 2, 3]),
("b1", &vec![4, 5, 7]), ("c1", &vec![7, 8, 9]),
);
let right = build_table(
("a2", &vec![10, 20, 30]),
("b1", &vec![4, 5, 6]),
("c2", &vec![70, 80, 90]),
);
let on = vec![(
Column::new_with_schema("b1", &left.schema())?,
Column::new_with_schema("b1", &right.schema())?,
)];
let (columns, batches) = join_collect(
left.clone(),
right.clone(),
on.clone(),
&JoinType::Left,
false,
task_ctx,
)
.await?;
assert_eq!(columns, vec!["a1", "b1", "c1", "a2", "b1", "c2"]);
let expected = [
"+----+----+----+----+----+----+",
"| a1 | b1 | c1 | a2 | b1 | c2 |",
"+----+----+----+----+----+----+",
"| 1 | 4 | 7 | 10 | 4 | 70 |",
"| 2 | 5 | 8 | 20 | 5 | 80 |",
"| 3 | 7 | 9 | | | |",
"+----+----+----+----+----+----+",
];
assert_batches_sorted_eq!(expected, &batches);
Ok(())
}
#[tokio::test]
async fn partitioned_join_left_one() -> Result<()> {
let task_ctx = Arc::new(TaskContext::default());
let left = build_table(
("a1", &vec![1, 2, 3]),
("b1", &vec![4, 5, 7]), ("c1", &vec![7, 8, 9]),
);
let right = build_table(
("a2", &vec![10, 20, 30]),
("b1", &vec![4, 5, 6]),
("c2", &vec![70, 80, 90]),
);
let on = vec![(
Column::new_with_schema("b1", &left.schema())?,
Column::new_with_schema("b1", &right.schema())?,
)];
let (columns, batches) = partitioned_join_collect(
left.clone(),
right.clone(),
on.clone(),
&JoinType::Left,
false,
task_ctx,
)
.await?;
assert_eq!(columns, vec!["a1", "b1", "c1", "a2", "b1", "c2"]);
let expected = [
"+----+----+----+----+----+----+",
"| a1 | b1 | c1 | a2 | b1 | c2 |",
"+----+----+----+----+----+----+",
"| 1 | 4 | 7 | 10 | 4 | 70 |",
"| 2 | 5 | 8 | 20 | 5 | 80 |",
"| 3 | 7 | 9 | | | |",
"+----+----+----+----+----+----+",
];
assert_batches_sorted_eq!(expected, &batches);
Ok(())
}
fn build_semi_anti_left_table() -> Arc<dyn ExecutionPlan> {
build_table(
("a1", &vec![1, 3, 5, 7, 9, 11, 13]),
("b1", &vec![1, 3, 5, 7, 8, 8, 10]),
("c1", &vec![10, 30, 50, 70, 90, 110, 130]),
)
}
fn build_semi_anti_right_table() -> Arc<dyn ExecutionPlan> {
build_table(
("a2", &vec![8, 12, 6, 2, 10, 4]),
("b2", &vec![8, 10, 6, 2, 10, 4]),
("c2", &vec![20, 40, 60, 80, 100, 120]),
)
}
#[tokio::test]
async fn join_left_semi() -> Result<()> {
let task_ctx = Arc::new(TaskContext::default());
let left = build_semi_anti_left_table();
let right = build_semi_anti_right_table();
let on = vec![(
Column::new_with_schema("b1", &left.schema())?,
Column::new_with_schema("b2", &right.schema())?,
)];
let join = join(left, right, on, &JoinType::LeftSemi, false)?;
let columns = columns(&join.schema());
assert_eq!(columns, vec!["a1", "b1", "c1"]);
let stream = join.execute(0, task_ctx)?;
let batches = common::collect(stream).await?;
let expected = [
"+----+----+-----+",
"| a1 | b1 | c1 |",
"+----+----+-----+",
"| 11 | 8 | 110 |",
"| 13 | 10 | 130 |",
"| 9 | 8 | 90 |",
"+----+----+-----+",
];
assert_batches_sorted_eq!(expected, &batches);
Ok(())
}
#[tokio::test]
async fn join_left_semi_with_filter() -> Result<()> {
let task_ctx = Arc::new(TaskContext::default());
let left = build_semi_anti_left_table();
let right = build_semi_anti_right_table();
let on = vec![(
Column::new_with_schema("b1", &left.schema())?,
Column::new_with_schema("b2", &right.schema())?,
)];
let column_indices = vec![ColumnIndex {
index: 0,
side: JoinSide::Right,
}];
let intermediate_schema =
Schema::new(vec![Field::new("x", DataType::Int32, true)]);
let filter_expression = Arc::new(BinaryExpr::new(
Arc::new(Column::new("x", 0)),
Operator::NotEq,
Arc::new(Literal::new(ScalarValue::Int32(Some(10)))),
)) as Arc<dyn PhysicalExpr>;
let filter = JoinFilter::new(
filter_expression,
column_indices.clone(),
intermediate_schema.clone(),
);
let join = join_with_filter(
left.clone(),
right.clone(),
on.clone(),
filter,
&JoinType::LeftSemi,
false,
)?;
let columns_header = columns(&join.schema());
assert_eq!(columns_header.clone(), vec!["a1", "b1", "c1"]);
let stream = join.execute(0, task_ctx.clone())?;
let batches = common::collect(stream).await?;
let expected = [
"+----+----+-----+",
"| a1 | b1 | c1 |",
"+----+----+-----+",
"| 11 | 8 | 110 |",
"| 13 | 10 | 130 |",
"| 9 | 8 | 90 |",
"+----+----+-----+",
];
assert_batches_sorted_eq!(expected, &batches);
let filter_expression = Arc::new(BinaryExpr::new(
Arc::new(Column::new("x", 0)),
Operator::Gt,
Arc::new(Literal::new(ScalarValue::Int32(Some(10)))),
)) as Arc<dyn PhysicalExpr>;
let filter =
JoinFilter::new(filter_expression, column_indices, intermediate_schema);
let join = join_with_filter(left, right, on, filter, &JoinType::LeftSemi, false)?;
let columns_header = columns(&join.schema());
assert_eq!(columns_header, vec!["a1", "b1", "c1"]);
let stream = join.execute(0, task_ctx)?;
let batches = common::collect(stream).await?;
let expected = [
"+----+----+-----+",
"| a1 | b1 | c1 |",
"+----+----+-----+",
"| 13 | 10 | 130 |",
"+----+----+-----+",
];
assert_batches_sorted_eq!(expected, &batches);
Ok(())
}
#[tokio::test]
async fn join_right_semi() -> Result<()> {
let task_ctx = Arc::new(TaskContext::default());
let left = build_semi_anti_left_table();
let right = build_semi_anti_right_table();
let on = vec![(
Column::new_with_schema("b1", &left.schema())?,
Column::new_with_schema("b2", &right.schema())?,
)];
let join = join(left, right, on, &JoinType::RightSemi, false)?;
let columns = columns(&join.schema());
assert_eq!(columns, vec!["a2", "b2", "c2"]);
let stream = join.execute(0, task_ctx)?;
let batches = common::collect(stream).await?;
let expected = [
"+----+----+-----+",
"| a2 | b2 | c2 |",
"+----+----+-----+",
"| 10 | 10 | 100 |",
"| 12 | 10 | 40 |",
"| 8 | 8 | 20 |",
"+----+----+-----+",
];
assert_batches_sorted_eq!(expected, &batches);
Ok(())
}
#[tokio::test]
async fn join_right_semi_with_filter() -> Result<()> {
let task_ctx = Arc::new(TaskContext::default());
let left = build_semi_anti_left_table();
let right = build_semi_anti_right_table();
let on = vec![(
Column::new_with_schema("b1", &left.schema())?,
Column::new_with_schema("b2", &right.schema())?,
)];
let column_indices = vec![ColumnIndex {
index: 0,
side: JoinSide::Left,
}];
let intermediate_schema =
Schema::new(vec![Field::new("x", DataType::Int32, true)]);
let filter_expression = Arc::new(BinaryExpr::new(
Arc::new(Column::new("x", 0)),
Operator::NotEq,
Arc::new(Literal::new(ScalarValue::Int32(Some(9)))),
)) as Arc<dyn PhysicalExpr>;
let filter = JoinFilter::new(
filter_expression,
column_indices.clone(),
intermediate_schema.clone(),
);
let join = join_with_filter(
left.clone(),
right.clone(),
on.clone(),
filter,
&JoinType::RightSemi,
false,
)?;
let columns = columns(&join.schema());
assert_eq!(columns, vec!["a2", "b2", "c2"]);
let stream = join.execute(0, task_ctx.clone())?;
let batches = common::collect(stream).await?;
let expected = [
"+----+----+-----+",
"| a2 | b2 | c2 |",
"+----+----+-----+",
"| 10 | 10 | 100 |",
"| 12 | 10 | 40 |",
"| 8 | 8 | 20 |",
"+----+----+-----+",
];
assert_batches_sorted_eq!(expected, &batches);
let filter_expression = Arc::new(BinaryExpr::new(
Arc::new(Column::new("x", 0)),
Operator::Gt,
Arc::new(Literal::new(ScalarValue::Int32(Some(11)))),
)) as Arc<dyn PhysicalExpr>;
let filter =
JoinFilter::new(filter_expression, column_indices, intermediate_schema);
let join =
join_with_filter(left, right, on, filter, &JoinType::RightSemi, false)?;
let stream = join.execute(0, task_ctx)?;
let batches = common::collect(stream).await?;
let expected = [
"+----+----+-----+",
"| a2 | b2 | c2 |",
"+----+----+-----+",
"| 10 | 10 | 100 |",
"| 12 | 10 | 40 |",
"+----+----+-----+",
];
assert_batches_sorted_eq!(expected, &batches);
Ok(())
}
#[tokio::test]
async fn join_left_anti() -> Result<()> {
let task_ctx = Arc::new(TaskContext::default());
let left = build_semi_anti_left_table();
let right = build_semi_anti_right_table();
let on = vec![(
Column::new_with_schema("b1", &left.schema())?,
Column::new_with_schema("b2", &right.schema())?,
)];
let join = join(left, right, on, &JoinType::LeftAnti, false)?;
let columns = columns(&join.schema());
assert_eq!(columns, vec!["a1", "b1", "c1"]);
let stream = join.execute(0, task_ctx)?;
let batches = common::collect(stream).await?;
let expected = [
"+----+----+----+",
"| a1 | b1 | c1 |",
"+----+----+----+",
"| 1 | 1 | 10 |",
"| 3 | 3 | 30 |",
"| 5 | 5 | 50 |",
"| 7 | 7 | 70 |",
"+----+----+----+",
];
assert_batches_sorted_eq!(expected, &batches);
Ok(())
}
#[tokio::test]
async fn join_left_anti_with_filter() -> Result<()> {
let task_ctx = Arc::new(TaskContext::default());
let left = build_semi_anti_left_table();
let right = build_semi_anti_right_table();
let on = vec![(
Column::new_with_schema("b1", &left.schema())?,
Column::new_with_schema("b2", &right.schema())?,
)];
let column_indices = vec![ColumnIndex {
index: 0,
side: JoinSide::Right,
}];
let intermediate_schema =
Schema::new(vec![Field::new("x", DataType::Int32, true)]);
let filter_expression = Arc::new(BinaryExpr::new(
Arc::new(Column::new("x", 0)),
Operator::NotEq,
Arc::new(Literal::new(ScalarValue::Int32(Some(8)))),
)) as Arc<dyn PhysicalExpr>;
let filter = JoinFilter::new(
filter_expression,
column_indices.clone(),
intermediate_schema.clone(),
);
let join = join_with_filter(
left.clone(),
right.clone(),
on.clone(),
filter,
&JoinType::LeftAnti,
false,
)?;
let columns_header = columns(&join.schema());
assert_eq!(columns_header, vec!["a1", "b1", "c1"]);
let stream = join.execute(0, task_ctx.clone())?;
let batches = common::collect(stream).await?;
let expected = [
"+----+----+-----+",
"| a1 | b1 | c1 |",
"+----+----+-----+",
"| 1 | 1 | 10 |",
"| 11 | 8 | 110 |",
"| 3 | 3 | 30 |",
"| 5 | 5 | 50 |",
"| 7 | 7 | 70 |",
"| 9 | 8 | 90 |",
"+----+----+-----+",
];
assert_batches_sorted_eq!(expected, &batches);
let filter_expression = Arc::new(BinaryExpr::new(
Arc::new(Column::new("x", 0)),
Operator::NotEq,
Arc::new(Literal::new(ScalarValue::Int32(Some(8)))),
)) as Arc<dyn PhysicalExpr>;
let filter =
JoinFilter::new(filter_expression, column_indices, intermediate_schema);
let join = join_with_filter(left, right, on, filter, &JoinType::LeftAnti, false)?;
let columns_header = columns(&join.schema());
assert_eq!(columns_header, vec!["a1", "b1", "c1"]);
let stream = join.execute(0, task_ctx)?;
let batches = common::collect(stream).await?;
let expected = [
"+----+----+-----+",
"| a1 | b1 | c1 |",
"+----+----+-----+",
"| 1 | 1 | 10 |",
"| 11 | 8 | 110 |",
"| 3 | 3 | 30 |",
"| 5 | 5 | 50 |",
"| 7 | 7 | 70 |",
"| 9 | 8 | 90 |",
"+----+----+-----+",
];
assert_batches_sorted_eq!(expected, &batches);
Ok(())
}
#[tokio::test]
async fn join_right_anti() -> Result<()> {
let task_ctx = Arc::new(TaskContext::default());
let left = build_semi_anti_left_table();
let right = build_semi_anti_right_table();
let on = vec![(
Column::new_with_schema("b1", &left.schema())?,
Column::new_with_schema("b2", &right.schema())?,
)];
let join = join(left, right, on, &JoinType::RightAnti, false)?;
let columns = columns(&join.schema());
assert_eq!(columns, vec!["a2", "b2", "c2"]);
let stream = join.execute(0, task_ctx)?;
let batches = common::collect(stream).await?;
let expected = [
"+----+----+-----+",
"| a2 | b2 | c2 |",
"+----+----+-----+",
"| 2 | 2 | 80 |",
"| 4 | 4 | 120 |",
"| 6 | 6 | 60 |",
"+----+----+-----+",
];
assert_batches_sorted_eq!(expected, &batches);
Ok(())
}
#[tokio::test]
async fn join_right_anti_with_filter() -> Result<()> {
let task_ctx = Arc::new(TaskContext::default());
let left = build_semi_anti_left_table();
let right = build_semi_anti_right_table();
let on = vec![(
Column::new_with_schema("b1", &left.schema())?,
Column::new_with_schema("b2", &right.schema())?,
)];
let column_indices = vec![ColumnIndex {
index: 0,
side: JoinSide::Left,
}];
let intermediate_schema =
Schema::new(vec![Field::new("x", DataType::Int32, true)]);
let filter_expression = Arc::new(BinaryExpr::new(
Arc::new(Column::new("x", 0)),
Operator::NotEq,
Arc::new(Literal::new(ScalarValue::Int32(Some(13)))),
)) as Arc<dyn PhysicalExpr>;
let filter = JoinFilter::new(
filter_expression,
column_indices,
intermediate_schema.clone(),
);
let join = join_with_filter(
left.clone(),
right.clone(),
on.clone(),
filter,
&JoinType::RightAnti,
false,
)?;
let columns_header = columns(&join.schema());
assert_eq!(columns_header, vec!["a2", "b2", "c2"]);
let stream = join.execute(0, task_ctx.clone())?;
let batches = common::collect(stream).await?;
let expected = [
"+----+----+-----+",
"| a2 | b2 | c2 |",
"+----+----+-----+",
"| 10 | 10 | 100 |",
"| 12 | 10 | 40 |",
"| 2 | 2 | 80 |",
"| 4 | 4 | 120 |",
"| 6 | 6 | 60 |",
"+----+----+-----+",
];
assert_batches_sorted_eq!(expected, &batches);
let column_indices = vec![ColumnIndex {
index: 1,
side: JoinSide::Right,
}];
let filter_expression = Arc::new(BinaryExpr::new(
Arc::new(Column::new("x", 0)),
Operator::NotEq,
Arc::new(Literal::new(ScalarValue::Int32(Some(8)))),
)) as Arc<dyn PhysicalExpr>;
let filter =
JoinFilter::new(filter_expression, column_indices, intermediate_schema);
let join =
join_with_filter(left, right, on, filter, &JoinType::RightAnti, false)?;
let columns_header = columns(&join.schema());
assert_eq!(columns_header, vec!["a2", "b2", "c2"]);
let stream = join.execute(0, task_ctx)?;
let batches = common::collect(stream).await?;
let expected = [
"+----+----+-----+",
"| a2 | b2 | c2 |",
"+----+----+-----+",
"| 2 | 2 | 80 |",
"| 4 | 4 | 120 |",
"| 6 | 6 | 60 |",
"| 8 | 8 | 20 |",
"+----+----+-----+",
];
assert_batches_sorted_eq!(expected, &batches);
Ok(())
}
#[tokio::test]
async fn join_right_one() -> Result<()> {
let task_ctx = Arc::new(TaskContext::default());
let left = build_table(
("a1", &vec![1, 2, 3]),
("b1", &vec![4, 5, 7]),
("c1", &vec![7, 8, 9]),
);
let right = build_table(
("a2", &vec![10, 20, 30]),
("b1", &vec![4, 5, 6]), ("c2", &vec![70, 80, 90]),
);
let on = vec![(
Column::new_with_schema("b1", &left.schema())?,
Column::new_with_schema("b1", &right.schema())?,
)];
let (columns, batches) =
join_collect(left, right, on, &JoinType::Right, false, task_ctx).await?;
assert_eq!(columns, vec!["a1", "b1", "c1", "a2", "b1", "c2"]);
let expected = [
"+----+----+----+----+----+----+",
"| a1 | b1 | c1 | a2 | b1 | c2 |",
"+----+----+----+----+----+----+",
"| | | | 30 | 6 | 90 |",
"| 1 | 4 | 7 | 10 | 4 | 70 |",
"| 2 | 5 | 8 | 20 | 5 | 80 |",
"+----+----+----+----+----+----+",
];
assert_batches_sorted_eq!(expected, &batches);
Ok(())
}
#[tokio::test]
async fn partitioned_join_right_one() -> Result<()> {
let task_ctx = Arc::new(TaskContext::default());
let left = build_table(
("a1", &vec![1, 2, 3]),
("b1", &vec![4, 5, 7]),
("c1", &vec![7, 8, 9]),
);
let right = build_table(
("a2", &vec![10, 20, 30]),
("b1", &vec![4, 5, 6]), ("c2", &vec![70, 80, 90]),
);
let on = vec![(
Column::new_with_schema("b1", &left.schema())?,
Column::new_with_schema("b1", &right.schema())?,
)];
let (columns, batches) =
partitioned_join_collect(left, right, on, &JoinType::Right, false, task_ctx)
.await?;
assert_eq!(columns, vec!["a1", "b1", "c1", "a2", "b1", "c2"]);
let expected = [
"+----+----+----+----+----+----+",
"| a1 | b1 | c1 | a2 | b1 | c2 |",
"+----+----+----+----+----+----+",
"| | | | 30 | 6 | 90 |",
"| 1 | 4 | 7 | 10 | 4 | 70 |",
"| 2 | 5 | 8 | 20 | 5 | 80 |",
"+----+----+----+----+----+----+",
];
assert_batches_sorted_eq!(expected, &batches);
Ok(())
}
#[tokio::test]
async fn join_full_one() -> Result<()> {
let task_ctx = Arc::new(TaskContext::default());
let left = build_table(
("a1", &vec![1, 2, 3]),
("b1", &vec![4, 5, 7]), ("c1", &vec![7, 8, 9]),
);
let right = build_table(
("a2", &vec![10, 20, 30]),
("b2", &vec![4, 5, 6]),
("c2", &vec![70, 80, 90]),
);
let on = vec![(
Column::new_with_schema("b1", &left.schema()).unwrap(),
Column::new_with_schema("b2", &right.schema()).unwrap(),
)];
let join = join(left, right, on, &JoinType::Full, false)?;
let columns = columns(&join.schema());
assert_eq!(columns, vec!["a1", "b1", "c1", "a2", "b2", "c2"]);
let stream = join.execute(0, task_ctx)?;
let batches = common::collect(stream).await?;
let expected = [
"+----+----+----+----+----+----+",
"| a1 | b1 | c1 | a2 | b2 | c2 |",
"+----+----+----+----+----+----+",
"| | | | 30 | 6 | 90 |",
"| 1 | 4 | 7 | 10 | 4 | 70 |",
"| 2 | 5 | 8 | 20 | 5 | 80 |",
"| 3 | 7 | 9 | | | |",
"+----+----+----+----+----+----+",
];
assert_batches_sorted_eq!(expected, &batches);
Ok(())
}
#[test]
fn join_with_hash_collision() -> Result<()> {
let mut hashmap_left = RawTable::with_capacity(2);
let left = build_table_i32(
("a", &vec![10, 20]),
("x", &vec![100, 200]),
("y", &vec![200, 300]),
);
let random_state = RandomState::with_seeds(0, 0, 0, 0);
let hashes_buff = &mut vec![0; left.num_rows()];
let hashes =
create_hashes(&[left.columns()[0].clone()], &random_state, hashes_buff)?;
hashmap_left.insert(hashes[0], (hashes[0], 1), |(h, _)| *h);
hashmap_left.insert(hashes[1], (hashes[1], 1), |(h, _)| *h);
let next = vec![2, 0];
let right = build_table_i32(
("a", &vec![10, 20]),
("b", &vec![0, 0]),
("c", &vec![30, 40]),
);
let left_data = (
JoinHashMap {
map: hashmap_left,
next,
},
left,
);
let (l, r) = build_equal_condition_join_indices(
&left_data.0,
&left_data.1,
&right,
&[Column::new("a", 0)],
&[Column::new("a", 0)],
&random_state,
false,
&mut vec![0; right.num_rows()],
None,
JoinSide::Left,
None,
)?;
let mut left_ids = UInt64Builder::with_capacity(0);
left_ids.append_value(0);
left_ids.append_value(1);
let mut right_ids = UInt32Builder::with_capacity(0);
right_ids.append_value(0);
right_ids.append_value(1);
assert_eq!(left_ids.finish(), l);
assert_eq!(right_ids.finish(), r);
Ok(())
}
#[tokio::test]
async fn join_with_duplicated_column_names() -> Result<()> {
let task_ctx = Arc::new(TaskContext::default());
let left = build_table(
("a", &vec![1, 2, 3]),
("b", &vec![4, 5, 7]),
("c", &vec![7, 8, 9]),
);
let right = build_table(
("a", &vec![10, 20, 30]),
("b", &vec![1, 2, 7]),
("c", &vec![70, 80, 90]),
);
let on = vec![(
Column::new_with_schema("a", &left.schema()).unwrap(),
Column::new_with_schema("b", &right.schema()).unwrap(),
)];
let join = join(left, right, on, &JoinType::Inner, false)?;
let columns = columns(&join.schema());
assert_eq!(columns, vec!["a", "b", "c", "a", "b", "c"]);
let stream = join.execute(0, task_ctx)?;
let batches = common::collect(stream).await?;
let expected = [
"+---+---+---+----+---+----+",
"| a | b | c | a | b | c |",
"+---+---+---+----+---+----+",
"| 1 | 4 | 7 | 10 | 1 | 70 |",
"| 2 | 5 | 8 | 20 | 2 | 80 |",
"+---+---+---+----+---+----+",
];
assert_batches_sorted_eq!(expected, &batches);
Ok(())
}
fn prepare_join_filter() -> JoinFilter {
let column_indices = vec![
ColumnIndex {
index: 2,
side: JoinSide::Left,
},
ColumnIndex {
index: 2,
side: JoinSide::Right,
},
];
let intermediate_schema = Schema::new(vec![
Field::new("c", DataType::Int32, true),
Field::new("c", DataType::Int32, true),
]);
let filter_expression = Arc::new(BinaryExpr::new(
Arc::new(Column::new("c", 0)),
Operator::Gt,
Arc::new(Column::new("c", 1)),
)) as Arc<dyn PhysicalExpr>;
JoinFilter::new(filter_expression, column_indices, intermediate_schema)
}
#[tokio::test]
async fn join_inner_with_filter() -> Result<()> {
let task_ctx = Arc::new(TaskContext::default());
let left = build_table(
("a", &vec![0, 1, 2, 2]),
("b", &vec![4, 5, 7, 8]),
("c", &vec![7, 8, 9, 1]),
);
let right = build_table(
("a", &vec![10, 20, 30, 40]),
("b", &vec![2, 2, 3, 4]),
("c", &vec![7, 5, 6, 4]),
);
let on = vec![(
Column::new_with_schema("a", &left.schema()).unwrap(),
Column::new_with_schema("b", &right.schema()).unwrap(),
)];
let filter = prepare_join_filter();
let join = join_with_filter(left, right, on, filter, &JoinType::Inner, false)?;
let columns = columns(&join.schema());
assert_eq!(columns, vec!["a", "b", "c", "a", "b", "c"]);
let stream = join.execute(0, task_ctx)?;
let batches = common::collect(stream).await?;
let expected = [
"+---+---+---+----+---+---+",
"| a | b | c | a | b | c |",
"+---+---+---+----+---+---+",
"| 2 | 7 | 9 | 10 | 2 | 7 |",
"| 2 | 7 | 9 | 20 | 2 | 5 |",
"+---+---+---+----+---+---+",
];
assert_batches_sorted_eq!(expected, &batches);
Ok(())
}
#[tokio::test]
async fn join_left_with_filter() -> Result<()> {
let task_ctx = Arc::new(TaskContext::default());
let left = build_table(
("a", &vec![0, 1, 2, 2]),
("b", &vec![4, 5, 7, 8]),
("c", &vec![7, 8, 9, 1]),
);
let right = build_table(
("a", &vec![10, 20, 30, 40]),
("b", &vec![2, 2, 3, 4]),
("c", &vec![7, 5, 6, 4]),
);
let on = vec![(
Column::new_with_schema("a", &left.schema()).unwrap(),
Column::new_with_schema("b", &right.schema()).unwrap(),
)];
let filter = prepare_join_filter();
let join = join_with_filter(left, right, on, filter, &JoinType::Left, false)?;
let columns = columns(&join.schema());
assert_eq!(columns, vec!["a", "b", "c", "a", "b", "c"]);
let stream = join.execute(0, task_ctx)?;
let batches = common::collect(stream).await?;
let expected = [
"+---+---+---+----+---+---+",
"| a | b | c | a | b | c |",
"+---+---+---+----+---+---+",
"| 0 | 4 | 7 | | | |",
"| 1 | 5 | 8 | | | |",
"| 2 | 7 | 9 | 10 | 2 | 7 |",
"| 2 | 7 | 9 | 20 | 2 | 5 |",
"| 2 | 8 | 1 | | | |",
"+---+---+---+----+---+---+",
];
assert_batches_sorted_eq!(expected, &batches);
Ok(())
}
#[tokio::test]
async fn join_right_with_filter() -> Result<()> {
let task_ctx = Arc::new(TaskContext::default());
let left = build_table(
("a", &vec![0, 1, 2, 2]),
("b", &vec![4, 5, 7, 8]),
("c", &vec![7, 8, 9, 1]),
);
let right = build_table(
("a", &vec![10, 20, 30, 40]),
("b", &vec![2, 2, 3, 4]),
("c", &vec![7, 5, 6, 4]),
);
let on = vec![(
Column::new_with_schema("a", &left.schema()).unwrap(),
Column::new_with_schema("b", &right.schema()).unwrap(),
)];
let filter = prepare_join_filter();
let join = join_with_filter(left, right, on, filter, &JoinType::Right, false)?;
let columns = columns(&join.schema());
assert_eq!(columns, vec!["a", "b", "c", "a", "b", "c"]);
let stream = join.execute(0, task_ctx)?;
let batches = common::collect(stream).await?;
let expected = [
"+---+---+---+----+---+---+",
"| a | b | c | a | b | c |",
"+---+---+---+----+---+---+",
"| | | | 30 | 3 | 6 |",
"| | | | 40 | 4 | 4 |",
"| 2 | 7 | 9 | 10 | 2 | 7 |",
"| 2 | 7 | 9 | 20 | 2 | 5 |",
"+---+---+---+----+---+---+",
];
assert_batches_sorted_eq!(expected, &batches);
Ok(())
}
#[tokio::test]
async fn join_full_with_filter() -> Result<()> {
let task_ctx = Arc::new(TaskContext::default());
let left = build_table(
("a", &vec![0, 1, 2, 2]),
("b", &vec![4, 5, 7, 8]),
("c", &vec![7, 8, 9, 1]),
);
let right = build_table(
("a", &vec![10, 20, 30, 40]),
("b", &vec![2, 2, 3, 4]),
("c", &vec![7, 5, 6, 4]),
);
let on = vec![(
Column::new_with_schema("a", &left.schema()).unwrap(),
Column::new_with_schema("b", &right.schema()).unwrap(),
)];
let filter = prepare_join_filter();
let join = join_with_filter(left, right, on, filter, &JoinType::Full, false)?;
let columns = columns(&join.schema());
assert_eq!(columns, vec!["a", "b", "c", "a", "b", "c"]);
let stream = join.execute(0, task_ctx)?;
let batches = common::collect(stream).await?;
let expected = [
"+---+---+---+----+---+---+",
"| a | b | c | a | b | c |",
"+---+---+---+----+---+---+",
"| | | | 30 | 3 | 6 |",
"| | | | 40 | 4 | 4 |",
"| 2 | 7 | 9 | 10 | 2 | 7 |",
"| 2 | 7 | 9 | 20 | 2 | 5 |",
"| 0 | 4 | 7 | | | |",
"| 1 | 5 | 8 | | | |",
"| 2 | 8 | 1 | | | |",
"+---+---+---+----+---+---+",
];
assert_batches_sorted_eq!(expected, &batches);
Ok(())
}
#[tokio::test]
async fn join_date32() -> Result<()> {
let schema = Arc::new(Schema::new(vec![
Field::new("date", DataType::Date32, false),
Field::new("n", DataType::Int32, false),
]));
let dates: ArrayRef = Arc::new(Date32Array::from(vec![19107, 19108, 19109]));
let n: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 3]));
let batch = RecordBatch::try_new(schema.clone(), vec![dates, n])?;
let left =
Arc::new(MemoryExec::try_new(&[vec![batch]], schema.clone(), None).unwrap());
let dates: ArrayRef = Arc::new(Date32Array::from(vec![19108, 19108, 19109]));
let n: ArrayRef = Arc::new(Int32Array::from(vec![4, 5, 6]));
let batch = RecordBatch::try_new(schema.clone(), vec![dates, n])?;
let right = Arc::new(MemoryExec::try_new(&[vec![batch]], schema, None).unwrap());
let on = vec![(
Column::new_with_schema("date", &left.schema()).unwrap(),
Column::new_with_schema("date", &right.schema()).unwrap(),
)];
let join = join(left, right, on, &JoinType::Inner, false)?;
let task_ctx = Arc::new(TaskContext::default());
let stream = join.execute(0, task_ctx)?;
let batches = common::collect(stream).await?;
let expected = [
"+------------+---+------------+---+",
"| date | n | date | n |",
"+------------+---+------------+---+",
"| 2022-04-26 | 2 | 2022-04-26 | 4 |",
"| 2022-04-26 | 2 | 2022-04-26 | 5 |",
"| 2022-04-27 | 3 | 2022-04-27 | 6 |",
"+------------+---+------------+---+",
];
assert_batches_sorted_eq!(expected, &batches);
Ok(())
}
#[tokio::test]
async fn join_with_error_right() {
let left = build_table(
("a1", &vec![1, 2, 3]),
("b1", &vec![4, 5, 7]),
("c1", &vec![7, 8, 9]),
);
let err = exec_err!("bad data error");
let right = build_table_i32(("a2", &vec![]), ("b1", &vec![]), ("c2", &vec![]));
let on = vec![(
Column::new_with_schema("b1", &left.schema()).unwrap(),
Column::new_with_schema("b1", &right.schema()).unwrap(),
)];
let schema = right.schema();
let right = build_table_i32(("a2", &vec![]), ("b1", &vec![]), ("c2", &vec![]));
let right_input = Arc::new(MockExec::new(vec![Ok(right), err], schema));
let join_types = vec![
JoinType::Inner,
JoinType::Left,
JoinType::Right,
JoinType::Full,
JoinType::LeftSemi,
JoinType::LeftAnti,
JoinType::RightSemi,
JoinType::RightAnti,
];
for join_type in join_types {
let join = join(
left.clone(),
right_input.clone(),
on.clone(),
&join_type,
false,
)
.unwrap();
let task_ctx = Arc::new(TaskContext::default());
let stream = join.execute(0, task_ctx).unwrap();
let result_string = crate::common::collect(stream)
.await
.unwrap_err()
.to_string();
assert!(
result_string.contains("bad data error"),
"actual: {result_string}"
);
}
}
#[tokio::test]
async fn single_partition_join_overallocation() -> Result<()> {
let left = build_table(
("a1", &vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 0]),
("b1", &vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 0]),
("c1", &vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 0]),
);
let right = build_table(
("a2", &vec![10, 11]),
("b2", &vec![12, 13]),
("c2", &vec![14, 15]),
);
let on = vec![(
Column::new_with_schema("a1", &left.schema()).unwrap(),
Column::new_with_schema("b2", &right.schema()).unwrap(),
)];
let join_types = vec![
JoinType::Inner,
JoinType::Left,
JoinType::Right,
JoinType::Full,
JoinType::LeftSemi,
JoinType::LeftAnti,
JoinType::RightSemi,
JoinType::RightAnti,
];
for join_type in join_types {
let runtime_config = RuntimeConfig::new().with_memory_limit(100, 1.0);
let runtime = Arc::new(RuntimeEnv::new(runtime_config)?);
let task_ctx = TaskContext::default().with_runtime(runtime);
let task_ctx = Arc::new(task_ctx);
let join = join(left.clone(), right.clone(), on.clone(), &join_type, false)?;
let stream = join.execute(0, task_ctx)?;
let err = common::collect(stream).await.unwrap_err();
assert_contains!(
err.to_string(),
"External error: Resources exhausted: Failed to allocate additional"
);
assert_contains!(err.to_string(), "HashJoinInput");
}
Ok(())
}
#[tokio::test]
async fn partitioned_join_overallocation() -> Result<()> {
let left_batch = build_table_i32(
("a1", &vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 0]),
("b1", &vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 0]),
("c1", &vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 0]),
);
let left = Arc::new(
MemoryExec::try_new(
&[vec![left_batch.clone()], vec![left_batch.clone()]],
left_batch.schema(),
None,
)
.unwrap(),
);
let right_batch = build_table_i32(
("a2", &vec![10, 11]),
("b2", &vec![12, 13]),
("c2", &vec![14, 15]),
);
let right = Arc::new(
MemoryExec::try_new(
&[vec![right_batch.clone()], vec![right_batch.clone()]],
right_batch.schema(),
None,
)
.unwrap(),
);
let on = vec![(
Column::new_with_schema("b1", &left_batch.schema())?,
Column::new_with_schema("b2", &right_batch.schema())?,
)];
let join_types = vec![
JoinType::Inner,
JoinType::Left,
JoinType::Right,
JoinType::Full,
JoinType::LeftSemi,
JoinType::LeftAnti,
JoinType::RightSemi,
JoinType::RightAnti,
];
for join_type in join_types {
let runtime_config = RuntimeConfig::new().with_memory_limit(100, 1.0);
let runtime = Arc::new(RuntimeEnv::new(runtime_config)?);
let session_config = SessionConfig::default().with_batch_size(50);
let task_ctx = TaskContext::default()
.with_session_config(session_config)
.with_runtime(runtime);
let task_ctx = Arc::new(task_ctx);
let join = HashJoinExec::try_new(
left.clone(),
right.clone(),
on.clone(),
None,
&join_type,
PartitionMode::Partitioned,
false,
)?;
let stream = join.execute(1, task_ctx)?;
let err = common::collect(stream).await.unwrap_err();
assert_contains!(
err.to_string(),
"External error: Resources exhausted: Failed to allocate additional"
);
assert_contains!(err.to_string(), "HashJoinInput[1]");
}
Ok(())
}
fn columns(schema: &Schema) -> Vec<String> {
schema.fields().iter().map(|f| f.name().clone()).collect()
}
}