jonathanc-n commented on code in PR #16660: URL: https://github.com/apache/datafusion/pull/16660#discussion_r2274842425
########## datafusion/physical-plan/src/joins/piecewise_merge_join.rs: ########## @@ -0,0 +1,2071 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use arrow::array::{new_null_array, Array, RecordBatchOptions}; +use arrow::compute::take; +use arrow::{ + array::{ + ArrayRef, BooleanBufferBuilder, RecordBatch, UInt32Array, UInt32Builder, + UInt64Array, UInt64Builder, + }, + compute::{concat_batches, sort_to_indices, take_record_batch}, + util::bit_util, +}; +use arrow_schema::{ArrowError, Schema, SchemaRef, SortOptions}; +use datafusion_common::NullEquality; +use datafusion_common::{ + exec_err, internal_err, plan_err, utils::compare_rows, JoinSide, Result, ScalarValue, +}; +use datafusion_execution::{ + memory_pool::{MemoryConsumer, MemoryReservation}, + RecordBatchStream, SendableRecordBatchStream, +}; +use datafusion_expr::{JoinType, Operator}; +use datafusion_functions_aggregate_common::min_max::{max_batch, min_batch}; +use datafusion_physical_expr::equivalence::join_equivalence_properties; +use datafusion_physical_expr::{ + LexOrdering, OrderingRequirements, PhysicalExpr, PhysicalExprRef, PhysicalSortExpr, +}; +use datafusion_physical_expr_common::physical_expr::fmt_sql; +use futures::{Stream, StreamExt, TryStreamExt}; +use parking_lot::Mutex; +use std::fmt::Formatter; +use std::{cmp::Ordering, task::ready}; +use std::{sync::Arc, task::Poll}; + +use crate::execution_plan::{boundedness_from_children, EmissionType}; + +use crate::joins::sort_merge_join::compare_join_arrays; +use crate::joins::utils::{ + get_final_indices_from_shared_bitmap, symmetric_join_output_partitioning, +}; +use crate::{handle_state, DisplayAs, DisplayFormatType, ExecutionPlanProperties}; +use crate::{ + joins::{ + utils::{ + build_join_schema, BuildProbeJoinMetrics, OnceAsync, OnceFut, + StatefulStreamResult, + }, + SharedBitmapBuilder, + }, + metrics::ExecutionPlanMetricsSet, + spill::get_record_batch_memory_size, + ExecutionPlan, PlanProperties, +}; + +/// `PiecewiseMergeJoinExec` is a join execution plan that only evaluates single range filter. +/// +/// The physical planner will choose to evalute this join when there is only one range predicate. This +/// is a binary expression which contains [`Operator::Lt`], [`Operator::LtEq`], [`Operator::Gt`], and +/// [`Operator::GtEq`].: +/// Examples: +/// - `col0` < `colb`, `col0` <= `colb`, `col0` > `colb`, `col0` >= `colb` +/// +/// Since the join only support range predicates, equijoins are not supported in `PiecewiseMergeJoinExec`, +/// however you can first evaluate another join and run `PiecewiseMergeJoinExec` if left with one range +/// predicate. +/// +/// # Execution Plan Inputs +/// For `PiecewiseMergeJoin` we label all left inputs as the `streamed' side and the right outputs as the +/// 'buffered' side. +/// +/// `PiecewiseMergeJoin` takes a sorted input for the side to be buffered and is able to sort streamed record +/// batches during processing. Sorted input must specifically be ascending/descending based on the operator. +/// +/// # Algorithms +/// Classic joins are processed differently compared to existence joins. +/// +/// ## Classic Joins (Inner, Full, Left, Right) +/// For classic joins we buffer the right side (buffered), and incrementally process the left side (streamed). +/// Every streamed batch is sorted so we can perform a sort merge algorithm. For the buffered side we want to +/// have it already sorted either ascending or descending based on the operator as this allows us to emit all +/// the rows from a given point to the end as matches. Sorting the streamed side allows us to start the pointer +/// from the previous row's match on the buffered side. +/// +/// Here is an example: +/// +/// We perform a `JoinType::Left` with these two batches and the operator being `Operator::Lt`(<). For each +/// row on the streamed side we move a pointer on the buffered until it matches the condition. Once we reach +/// the row which matches (in this case with row 1 on streamed will have its first match on row 2 on +/// buffered; 100 < 200 is true), we can emit all rows after that match. We can emit the rows like this because +/// if the batch is sorted in ascending order, every subsequent row will also satisfy the condition as they will +/// all be larger values. +/// +/// ```text +/// SQL statement: +/// SELECT * +/// FROM (VALUES (100), (200), (500)) AS streamed(a) +/// LEFT JOIN (VALUES (100), (200), (200), (300), (400)) AS buffered(b) +/// ON streamed.a < buffered.b; +/// +/// Processing Row 1: +/// +/// Sorted Streamed Side Sorted Buffered Side +/// ┌──────────────────┐ ┌──────────────────┐ +/// 1 │ 100 │ 1 │ 100 │ +/// ├──────────────────┤ ├──────────────────┤ +/// 2 │ 200 │ 2 │ 200 │ ─┐ +/// ├──────────────────┤ ├──────────────────┤ │ For row 1 on streamed side with +/// 3 │ 500 │ 3 │ 200 │ │ value 100, we emit rows 2 - 5 +/// └──────────────────┘ ├──────────────────┤ │ as matches when the operator is +/// 4 │ 300 │ │ `Operator::Lt` (<) Emitting all +/// ├──────────────────┤ │ rows after the first match (row 2 +/// 5 │ 400 │ ─┘ buffered side; 100 < 200) +/// └──────────────────┘ +/// +/// Processing Row 2: +/// By sorting the streamed side we know +/// +/// Sorted Streamed Side Sorted Buffered Side +/// ┌──────────────────┐ ┌──────────────────┐ +/// 1 │ 100 │ 1 │ 100 │ +/// ├──────────────────┤ ├──────────────────┤ +/// 2 │ 200 │ 2 │ 200 │ <- Start here when probing for the streamed +/// ├──────────────────┤ ├──────────────────┤ side row 2. +/// 3 │ 500 │ 3 │ 200 │ +/// └──────────────────┘ ├──────────────────┤ +/// 4 │ 300 │ +/// ├──────────────────┤ +/// 5 │ 400 | +/// └──────────────────┘ +/// ``` +/// +/// ## Existence Joins (Semi, Anti, Mark) +/// Existence joins are made magnitudes of times faster with a `PiecewiseMergeJoin` as we only need to find +/// the min/max value of the streamed side to be able to emit all matches on the buffered side. By putting +/// the side we need to mark onto the sorted buffer side, we can emit all these matches at once. +/// +/// For Left Semi, Anti, and Mark joins we swap the inputs so that the marked side is on the buffered side. +/// +/// Here is an example: +/// We perform a `JoinType::LeftSemi` with these two batches and the operator being `Operator::Lt`(<). Because +/// the operator is `Operator::Lt` we can find the minimum value in the streamed side; in this case it is 200. +/// We can then advance a pointer from the start of the buffer side until we find the first value that satisfies +/// the predicate. All rows after that first matched value satisfy the condition 200 < x so we can mark all of +/// those rows as matched. +/// +/// ```text +/// SQL statement: +/// SELECT * +/// FROM (VALUES (100), (200), (500)) AS streamed(a) +/// LEFT SEMI JOIN (VALUES (100), (200), (200), (300), (400)) AS buffered(b) +/// ON streamed.a < buffered.b; +/// +/// Unsorted Streamed Side Sorted Buffered Side +/// ┌──────────────────┐ ┌──────────────────┐ +/// 1 │ 500 │ 1 │ 100 │ +/// ├──────────────────┤ ├──────────────────┤ +/// 2 │ 200 │ 2 │ 200 │ +/// ├──────────────────┤ ├──────────────────┤ +/// 3 │ 300 │ 3 │ 200 │ +/// └──────────────────┘ ├──────────────────┤ +/// min value: 200 4 │ 300 │ ─┐ +/// ├──────────────────┤ | We emit matches for row 4 - 5 on the +/// 5 │ 400 | ─┘ buffered side. +/// └──────────────────┘ +/// ``` +/// +/// For both types of joins, the buffered side must be sorted ascending for `Operator::Lt`(<) or +/// `Operator::LtEq`(<=) and descending for `Operator::Gt`(>) or `Operator::GtEq`(>=). +/// +/// # Further Reference Material +/// DuckDB blog on Range Joins: [Range Joins in DuckDB](https://duckdb.org/2022/05/27/iejoin.html) +#[derive(Debug)] +pub struct PiecewiseMergeJoinExec { + /// Left sorted joining execution plan + pub streamed: Arc<dyn ExecutionPlan>, + /// Right sorting joining execution plan + pub buffered: Arc<dyn ExecutionPlan>, + /// The two expressions being compared + pub on: (Arc<dyn PhysicalExpr>, Arc<dyn PhysicalExpr>), + /// Comparison operator in the range predicate + pub operator: Operator, + /// How the join is performed + pub join_type: JoinType, + /// The schema once the join is applied + schema: SchemaRef, + buffered_fut: OnceAsync<BufferedSideData>, + /// Execution metrics + metrics: ExecutionPlanMetricsSet, + /// The left SortExpr + left_sort_exprs: LexOrdering, + /// The right SortExpr + right_sort_exprs: LexOrdering, + /// Sort options of join columns used in sorting the stream and buffered execution plans + sort_options: SortOptions, + /// Cache holding plan properties like equivalences, output partitioning etc. + cache: PlanProperties, +} + +impl PiecewiseMergeJoinExec { + pub fn try_new( + streamed: Arc<dyn ExecutionPlan>, + buffered: Arc<dyn ExecutionPlan>, + on: (Arc<dyn PhysicalExpr>, Arc<dyn PhysicalExpr>), + operator: Operator, + join_type: JoinType, + ) -> Result<Self> { + // TODO: Implement mark joins for PiecewiseMergeJoin + if matches!(join_type, JoinType::LeftMark | JoinType::RightMark) { + return plan_err!( + "Mark Joins are currently not supported for PiecewiseMergeJoin" + ); + } + + // We take the operator and enforce a sort order on the streamed + buffered side based on + // the operator type. + let sort_options = match operator { + Operator::Lt | Operator::LtEq => { + // For the Left existence joins the inputs will be swapped so we need to switch the sort + // options. + if is_left_existence_join(join_type) { + SortOptions::new(true, false) + } else { + SortOptions::new(false, false) + } + } + Operator::Gt | Operator::GtEq => { + if is_left_existence_join(join_type) { + SortOptions::new(false, false) + } else { + SortOptions::new(true, false) + } + } + _ => { + return plan_err!( + "Cannot contain non-range operator in PiecewiseMergeJoinExec" + ) + } + }; + + let left_sort_exprs = + vec![PhysicalSortExpr::new(Arc::clone(&on.0), sort_options)]; + let right_sort_exprs = + vec![PhysicalSortExpr::new(Arc::clone(&on.1), sort_options)]; + let Some(left_sort_exprs) = LexOrdering::new(left_sort_exprs) else { + return plan_err!( + "PiecewiseMergeJoinExec requires valid sort expressions for its left side" + ); + }; + let Some(right_sort_exprs) = LexOrdering::new(right_sort_exprs) else { + return plan_err!( + "PiecewiseMergeJoinExec requires valid sort expressions for its right side" + ); + }; + + let streamed_schema = streamed.schema(); + let buffered_schema = buffered.schema(); + + // Create output schema for the join + let schema = + Arc::new(build_join_schema(&streamed_schema, &buffered_schema, &join_type).0); + let cache = Self::compute_properties( + &streamed, + &buffered, + Arc::clone(&schema), + join_type, + &on, + )?; + + Ok(Self { + streamed, + buffered, + on, + operator, + join_type, + schema, + buffered_fut: Default::default(), + metrics: ExecutionPlanMetricsSet::new(), + left_sort_exprs, + right_sort_exprs, + sort_options, + cache, + }) + } + + /// Refeference to streamed side execution plan + pub fn streamed(&self) -> &Arc<dyn ExecutionPlan> { + &self.streamed + } + + /// Refeerence to buffered side execution plan + pub fn buffered(&self) -> &Arc<dyn ExecutionPlan> { + &self.buffered + } + + /// Join type + pub fn join_type(&self) -> JoinType { + self.join_type + } + + /// Reference to sort options + pub fn sort_options(&self) -> &SortOptions { + &self.sort_options + } + + /// Get probe side (streameded side) for the PiecewiseMergeJoin + /// In current implementation, probe side is determined according to join type. + pub fn probe_side(join_type: &JoinType) -> JoinSide { + match join_type { + JoinType::Right + | JoinType::RightSemi + | JoinType::RightAnti + | JoinType::RightMark => JoinSide::Left, + JoinType::Inner + | JoinType::Left + | JoinType::Full + | JoinType::LeftAnti + | JoinType::LeftSemi + | JoinType::LeftMark => JoinSide::Right, + } + } + + pub fn compute_properties( + streamed: &Arc<dyn ExecutionPlan>, + buffered: &Arc<dyn ExecutionPlan>, + schema: SchemaRef, + join_type: JoinType, + join_on: &(PhysicalExprRef, PhysicalExprRef), + ) -> Result<PlanProperties> { + let eq_properties = join_equivalence_properties( + streamed.equivalence_properties().clone(), + buffered.equivalence_properties().clone(), + &join_type, + schema, + &Self::maintains_input_order(join_type), + Some(Self::probe_side(&join_type)), + &[join_on.clone()], + )?; + + let output_partitioning = + symmetric_join_output_partitioning(streamed, buffered, &join_type)?; + + Ok(PlanProperties::new( + eq_properties, + output_partitioning, + EmissionType::Incremental, + boundedness_from_children([streamed, buffered]), + )) + } + + fn maintains_input_order(join_type: JoinType) -> Vec<bool> { + match join_type { + // The existence side is expected to come in sorted + JoinType::LeftSemi | JoinType::LeftAnti | JoinType::LeftMark => { + vec![true, false] + } + JoinType::RightSemi | JoinType::RightAnti | JoinType::RightMark => { + vec![false, true] + } + // Left, Right, Full, Inner Join is not guaranteed to maintain + // input order as the streamed side will be sorted during + // execution for `PiecewiseMergeJoin` + _ => vec![false, false], + } + } + + // TODO: We implement this with the physical planner. + pub fn swap_inputs(&self) -> Result<Arc<dyn ExecutionPlan>> { + todo!() + } +} + +impl ExecutionPlan for PiecewiseMergeJoinExec { + fn name(&self) -> &str { + "PiecewiseMergeJoinExec" + } + + fn as_any(&self) -> &dyn std::any::Any { + self + } + + fn properties(&self) -> &PlanProperties { + &self.cache + } + + fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> { + vec![&self.streamed, &self.buffered] + } + + fn required_input_ordering(&self) -> Vec<Option<OrderingRequirements>> { + // Existence joins don't need to be sorted on one side. + if is_left_existence_join(self.join_type) { + // Left side needs to be sorted because this will be swapped to the + // buffered side + vec![ + Some(OrderingRequirements::from(self.left_sort_exprs.clone())), + None, + ] + } else { + // We sort the left side in memory, so we do not need to enforce any sorting + vec![ + None, + Some(OrderingRequirements::from(self.right_sort_exprs.clone())), + ] + } + } + + fn with_new_children( + self: Arc<Self>, + children: Vec<Arc<dyn ExecutionPlan>>, + ) -> Result<Arc<dyn ExecutionPlan>> { + match &children[..] { + [left, right] => Ok(Arc::new(PiecewiseMergeJoinExec::try_new( + Arc::clone(left), + Arc::clone(right), + self.on.clone(), + self.operator, + self.join_type, + )?)), + _ => internal_err!( + "PiecewiseMergeJoin should have 2 children, found {}", + children.len() + ), + } + } + + fn execute( + &self, + partition: usize, + context: Arc<datafusion_execution::TaskContext>, + ) -> Result<SendableRecordBatchStream> { + let on_streamed = Arc::clone(&self.on.0); + let on_buffered = Arc::clone(&self.on.1); + + // If the join type is either LeftSemi, LeftAnti, or LeftMark we will swap the inputs + // and sort ordering because we want the mark side to be the buffered side. + let (streamed, buffered, on_streamed, on_buffered, operator) = + if is_left_existence_join(self.join_type) { + ( + Arc::clone(&self.buffered), + Arc::clone(&self.streamed), + on_buffered, + on_streamed, + self.operator.swap().unwrap(), + ) + } else { + ( + Arc::clone(&self.streamed), + Arc::clone(&self.buffered), + on_streamed, + on_buffered, + self.operator, + ) + }; + + let metrics = BuildProbeJoinMetrics::new(0, &self.metrics); + let buffered_fut = self.buffered_fut.try_once(|| { + let reservation = MemoryConsumer::new("PiecewiseMergeJoinInput") + .register(context.memory_pool()); + let buffered_stream = buffered.execute(partition, Arc::clone(&context))?; + Ok(build_buffered_data( + buffered_stream, + Arc::clone(&on_buffered), + metrics.clone(), + reservation, + build_visited_indices_map(self.join_type), + )) + })?; + + let streamed = streamed.execute(partition, Arc::clone(&context))?; + let existence_join = is_existence_join(self.join_type()); + + Ok(Box::pin(PiecewiseMergeJoinStream::try_new( + Arc::clone(&self.schema), + on_streamed, + self.join_type, + operator, + streamed, + BufferedSide::Initial(BufferedSideInitialState { buffered_fut }), + if existence_join { + PiecewiseMergeJoinStreamState::FetchStreamBatch + } else { + PiecewiseMergeJoinStreamState::WaitBufferedSide + }, + existence_join, + self.sort_options, + metrics, + ))) + } +} + +impl DisplayAs for PiecewiseMergeJoinExec { + fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { + let on_str = format!( + "({} {} {})", + fmt_sql(self.on.0.as_ref()), + self.operator, + fmt_sql(self.on.1.as_ref()) + ); + + match t { + DisplayFormatType::Default | DisplayFormatType::Verbose => { + write!( + f, + "PiecewiseMergeJoin: operator={:?}, join_type={:?}, on={}", + self.operator, self.join_type, on_str + ) + } + + DisplayFormatType::TreeRender => { + writeln!(f, "operator={:?}", self.operator)?; + if self.join_type != JoinType::Inner { + writeln!(f, "join_type={:?}", self.join_type)?; + } + writeln!(f, "on={on_str}") + } + } + } +} + +// Returns boolean for whether the join is an existence join +fn is_existence_join(join_type: JoinType) -> bool { + matches!( + join_type, + JoinType::LeftAnti + | JoinType::RightAnti + | JoinType::LeftSemi + | JoinType::RightSemi + | JoinType::LeftMark + | JoinType::RightMark + ) +} + +// Returns boolean for whether the join is a left existence join +fn is_left_existence_join(join_type: JoinType) -> bool { + matches!( + join_type, + JoinType::LeftAnti | JoinType::LeftSemi | JoinType::LeftMark + ) +} + +// Returns boolean to check if the join type needs to record +// buffered side matches for classic joins +fn need_produce_result_in_final(join_type: JoinType) -> bool { + matches!(join_type, JoinType::Full | JoinType::Right) +} + +// Returns boolean for whether or not we need to build the buffered side +// bitmap for marking matched rows on the buffered side. +fn build_visited_indices_map(join_type: JoinType) -> bool { + matches!( + join_type, + JoinType::Full + | JoinType::Right Review Comment: Full + Right aren't existence join -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: github-unsubscr...@datafusion.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: github-unsubscr...@datafusion.apache.org For additional commands, e-mail: github-h...@datafusion.apache.org