jonathanc-n commented on code in PR #17482: URL: https://github.com/apache/datafusion/pull/17482#discussion_r2427691046
########## datafusion/physical-plan/src/joins/piecewise_merge_join/classic_join.rs: ########## @@ -0,0 +1,1533 @@ +// 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. + +//! Stream Implementation for PiecewiseMergeJoin's Classic Join (Left, Right, Full, Inner) + +use arrow::array::{new_null_array, Array, PrimitiveBuilder}; +use arrow::compute::{take, BatchCoalescer}; +use arrow::datatypes::UInt32Type; +use arrow::{ + array::{ArrayRef, RecordBatch, UInt32Array}, + compute::{sort_to_indices, take_record_batch}, +}; +use arrow_schema::{Schema, SchemaRef, SortOptions}; +use datafusion_common::NullEquality; +use datafusion_common::{exec_err, internal_err, Result}; +use datafusion_execution::{RecordBatchStream, SendableRecordBatchStream}; +use datafusion_expr::{JoinType, Operator}; +use datafusion_physical_expr::PhysicalExprRef; +use futures::{Stream, StreamExt}; +use std::sync::atomic::AtomicUsize; +use std::{cmp::Ordering, task::ready}; +use std::{sync::Arc, task::Poll}; + +use crate::handle_state; +use crate::joins::piecewise_merge_join::exec::{BufferedSide, BufferedSideReadyState}; +use crate::joins::piecewise_merge_join::utils::need_produce_result_in_final; +use crate::joins::utils::{compare_join_arrays, get_final_indices_from_shared_bitmap}; +use crate::joins::utils::{BuildProbeJoinMetrics, StatefulStreamResult}; + +pub(super) enum PiecewiseMergeJoinStreamState { + WaitBufferedSide, + FetchStreamBatch, + ProcessStreamBatch(StreamedBatch), + ExhaustedStreamSide, + Completed, +} + +impl PiecewiseMergeJoinStreamState { + // Grab mutable reference to the current stream batch + fn try_as_process_stream_batch_mut(&mut self) -> Result<&mut StreamedBatch> { + match self { + PiecewiseMergeJoinStreamState::ProcessStreamBatch(state) => Ok(state), + _ => internal_err!("Expected streamed batch in StreamBatch"), + } + } +} + +pub(super) struct StreamedBatch { + pub batch: RecordBatch, + values: Vec<ArrayRef>, +} + +impl StreamedBatch { + #[allow(dead_code)] + fn new(batch: RecordBatch, values: Vec<ArrayRef>) -> Self { + Self { batch, values } + } + + fn values(&self) -> &Vec<ArrayRef> { + &self.values + } +} + +pub(super) struct ClassicPWMJStream { + // Output schema of the `PiecewiseMergeJoin` + pub schema: Arc<Schema>, + + // Physical expression that is evaluated on the streamed side + // We do not need on_buffered as this is already evaluated when + // creating the buffered side which happens before initializing + // `PiecewiseMergeJoinStream` + pub on_streamed: PhysicalExprRef, + // Type of join + pub join_type: JoinType, + // Comparison operator + pub operator: Operator, + // Streamed batch + pub streamed: SendableRecordBatchStream, + // Streamed schema + streamed_schema: SchemaRef, + // Buffered side data + buffered_side: BufferedSide, + // Tracks the state of the `PiecewiseMergeJoin` + state: PiecewiseMergeJoinStreamState, + // Sort option for buffered and streamed side (specifies whether + // the sort is ascending or descending) + sort_option: SortOptions, + // Metrics for build + probe joins + join_metrics: BuildProbeJoinMetrics, + // Tracking incremental state for emitting record batches + batch_process_state: BatchProcessState, + // To synchronize when partition needs to finish + remaining_partitions: Arc<AtomicUsize>, +} + +impl RecordBatchStream for ClassicPWMJStream { + fn schema(&self) -> SchemaRef { + Arc::clone(&self.schema) + } +} + +// `PiecewiseMergeJoinStreamState` is separated into `WaitBufferedSide`, `FetchStreamBatch`, +// `ProcessStreamBatch`, `ExhaustedStreamSide` and `Completed`. +// +// Classic Joins +// 1. `WaitBufferedSide` - Load in the buffered side data into memory. +// 2. `FetchStreamBatch` - Fetch + sort incoming stream batches. We switch the state to +// `Completed` if there are are still remaining partitions to process. It is only switched to +// `ExhaustedStreamBatch` if all partitions have been processed. +// 3. `ProcessStreamBatch` - Compare stream batch row values against the buffered side data. +// 4. `ExhaustedStreamBatch` - If the join type is Left or Inner we will return state as +// `Completed` however for Full and Right we will need to process the unmatched buffered rows. +impl ClassicPWMJStream { + // Creates a new `PiecewiseMergeJoinStream` instance + #[allow(clippy::too_many_arguments)] + pub fn try_new( + schema: Arc<Schema>, + on_streamed: PhysicalExprRef, + join_type: JoinType, + operator: Operator, + streamed: SendableRecordBatchStream, + buffered_side: BufferedSide, + state: PiecewiseMergeJoinStreamState, + sort_option: SortOptions, + join_metrics: BuildProbeJoinMetrics, + batch_size: usize, + remaining_partitions: Arc<AtomicUsize>, + ) -> Self { + Self { + schema: Arc::clone(&schema), + on_streamed, + join_type, + operator, + streamed_schema: streamed.schema(), + streamed, + buffered_side, + state, + sort_option, + join_metrics, + batch_process_state: BatchProcessState::new(schema, batch_size), + remaining_partitions, + } + } + + fn poll_next_impl( + &mut self, + cx: &mut std::task::Context<'_>, + ) -> Poll<Option<Result<RecordBatch>>> { + loop { + return match self.state { + PiecewiseMergeJoinStreamState::WaitBufferedSide => { + handle_state!(ready!(self.collect_buffered_side(cx))) + } + PiecewiseMergeJoinStreamState::FetchStreamBatch => { + handle_state!(ready!(self.fetch_stream_batch(cx))) + } + PiecewiseMergeJoinStreamState::ProcessStreamBatch(_) => { + handle_state!(self.process_stream_batch()) + } + PiecewiseMergeJoinStreamState::ExhaustedStreamSide => { + handle_state!(self.process_unmatched_buffered_batch()) + } + PiecewiseMergeJoinStreamState::Completed => Poll::Ready(None), + }; + } + } + + // Collects buffered side data + fn collect_buffered_side( + &mut self, + cx: &mut std::task::Context<'_>, + ) -> Poll<Result<StatefulStreamResult<Option<RecordBatch>>>> { + let build_timer = self.join_metrics.build_time.timer(); + let buffered_data = ready!(self + .buffered_side + .try_as_initial_mut()? + .buffered_fut + .get_shared(cx))?; + build_timer.done(); + + // We will start fetching stream batches for classic joins + self.state = PiecewiseMergeJoinStreamState::FetchStreamBatch; + + self.buffered_side = + BufferedSide::Ready(BufferedSideReadyState { buffered_data }); + + Poll::Ready(Ok(StatefulStreamResult::Continue)) + } + + // Fetches incoming stream batches + fn fetch_stream_batch( + &mut self, + cx: &mut std::task::Context<'_>, + ) -> Poll<Result<StatefulStreamResult<Option<RecordBatch>>>> { + match ready!(self.streamed.poll_next_unpin(cx)) { + None => { + if self + .remaining_partitions + .fetch_sub(1, std::sync::atomic::Ordering::SeqCst) + == 1 + { + self.batch_process_state.reset(); + self.state = PiecewiseMergeJoinStreamState::ExhaustedStreamSide; + } else { + self.state = PiecewiseMergeJoinStreamState::Completed; + } + } + Some(Ok(batch)) => { + // Evaluate the streamed physical expression on the stream batch + let stream_values: ArrayRef = self + .on_streamed + .evaluate(&batch)? + .into_array(batch.num_rows())?; + + self.join_metrics.input_batches.add(1); + self.join_metrics.input_rows.add(batch.num_rows()); + + // Sort stream values and change the streamed record batch accordingly + let indices = sort_to_indices( + stream_values.as_ref(), + Some(self.sort_option), + None, + )?; + let stream_batch = take_record_batch(&batch, &indices)?; + let stream_values = take(stream_values.as_ref(), &indices, None)?; + + // Reset BatchProcessState before processing a new stream batch + self.batch_process_state.reset(); + self.state = + PiecewiseMergeJoinStreamState::ProcessStreamBatch(StreamedBatch { + batch: stream_batch, + values: vec![stream_values], + }); + } + Some(Err(err)) => return Poll::Ready(Err(err)), + }; + + Poll::Ready(Ok(StatefulStreamResult::Continue)) + } + + // Only classic join will call. This function will process stream batches and evaluate against + // the buffered side data. + fn process_stream_batch( + &mut self, + ) -> Result<StatefulStreamResult<Option<RecordBatch>>> { + let buffered_side = self.buffered_side.try_as_ready_mut()?; + let stream_batch = self.state.try_as_process_stream_batch_mut()?; + + if let Some(batch) = self + .batch_process_state + .output_batches + .next_completed_batch() + { + return Ok(StatefulStreamResult::Ready(Some(batch))); + } + + // Produce more work + let batch = resolve_classic_join( + buffered_side, + stream_batch, + Arc::clone(&self.schema), + self.operator, + self.sort_option, + self.join_type, + &mut self.batch_process_state, + )?; + + if !self.batch_process_state.continue_process { + // We finished scanning this stream batch. + self.batch_process_state + .output_batches + .finish_buffered_batch()?; + if let Some(b) = self + .batch_process_state + .output_batches + .next_completed_batch() + { + self.state = PiecewiseMergeJoinStreamState::FetchStreamBatch; + return Ok(StatefulStreamResult::Ready(Some(b))); + } + // Nothing pending; hand back whatever `resolve` returned (often empty) and move on. + self.state = PiecewiseMergeJoinStreamState::FetchStreamBatch; + return Ok(StatefulStreamResult::Ready(Some(batch))); + } + + Ok(StatefulStreamResult::Ready(Some(batch))) + } + + // Process remaining unmatched rows + fn process_unmatched_buffered_batch( + &mut self, + ) -> Result<StatefulStreamResult<Option<RecordBatch>>> { + // Return early for `JoinType::Right` and `JoinType::Inner` + if matches!(self.join_type, JoinType::Right | JoinType::Inner) { + self.state = PiecewiseMergeJoinStreamState::Completed; + return Ok(StatefulStreamResult::Ready(None)); + } + + if !self.batch_process_state.continue_process { + if let Some(batch) = self + .batch_process_state + .output_batches + .next_completed_batch() + { + return Ok(StatefulStreamResult::Ready(Some(batch))); + } + + self.batch_process_state + .output_batches + .finish_buffered_batch()?; + if let Some(batch) = self + .batch_process_state + .output_batches + .next_completed_batch() + { + self.state = PiecewiseMergeJoinStreamState::Completed; + return Ok(StatefulStreamResult::Ready(Some(batch))); + } + } + + let buffered_data = + Arc::clone(&self.buffered_side.try_as_ready().unwrap().buffered_data); + + let (buffered_indices, _streamed_indices) = get_final_indices_from_shared_bitmap( + &buffered_data.visited_indices_bitmap, + self.join_type, + true, + ); + + let new_buffered_batch = + take_record_batch(buffered_data.batch(), &buffered_indices)?; + let mut buffered_columns = new_buffered_batch.columns().to_vec(); + + let streamed_columns: Vec<ArrayRef> = self + .streamed_schema + .fields() + .iter() + .map(|f| new_null_array(f.data_type(), new_buffered_batch.num_rows())) + .collect(); + + buffered_columns.extend(streamed_columns); + + let batch = RecordBatch::try_new(Arc::clone(&self.schema), buffered_columns)?; + + self.batch_process_state.output_batches.push_batch(batch)?; + + self.batch_process_state.continue_process = false; + if let Some(batch) = self + .batch_process_state + .output_batches + .next_completed_batch() + { + return Ok(StatefulStreamResult::Ready(Some(batch))); + } + + self.batch_process_state + .output_batches + .finish_buffered_batch()?; + if let Some(batch) = self + .batch_process_state + .output_batches + .next_completed_batch() + { + self.state = PiecewiseMergeJoinStreamState::Completed; + return Ok(StatefulStreamResult::Ready(Some(batch))); + } + + self.state = PiecewiseMergeJoinStreamState::Completed; + self.batch_process_state.reset(); + Ok(StatefulStreamResult::Ready(None)) + } +} + +struct BatchProcessState { + // Used to pick up from the last index on the stream side + output_batches: Box<BatchCoalescer>, + // Used to store the unmatched stream indices for `JoinType::Right` and `JoinType::Full` + unmatched_indices: PrimitiveBuilder<UInt32Type>, + // Used to store the start index on the buffered side; used to resume processing on the correct + // row + start_buffer_idx: usize, + // Used to store the start index on the stream side; used to resume processing on the correct + // row + start_stream_idx: usize, + // Signals if we found a match for the current stream row + found: bool, + // Signals to continue processing the current stream batch + continue_process: bool, +} + +impl BatchProcessState { + pub(crate) fn new(schema: Arc<Schema>, batch_size: usize) -> Self { + Self { + output_batches: Box::new(BatchCoalescer::new(schema, batch_size)), + unmatched_indices: PrimitiveBuilder::new(), + start_buffer_idx: 0, + start_stream_idx: 0, + found: false, + continue_process: true, + } + } + + pub(crate) fn reset(&mut self) { Review Comment: I don't think so, reset() gets called even if batch coalescer still has batches because everything isnt fully flushed until the end. -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
