jonathanc-n commented on code in PR #17482: URL: https://github.com/apache/datafusion/pull/17482#discussion_r2350515225
########## datafusion/physical-plan/src/joins/piecewise_merge_join/classic_join.rs: ########## @@ -0,0 +1,1471 @@ +// 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, PrimitiveArray, PrimitiveBuilder, RecordBatchOptions, +}; +use arrow::compute::take; +use arrow::datatypes::{UInt32Type, UInt64Type}; +use arrow::{ + array::{ + ArrayRef, RecordBatch, UInt32Array, UInt32Builder, UInt64Array, UInt64Builder, + }, + compute::{sort_to_indices, take_record_batch}, +}; +use arrow_schema::{ArrowError, 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::{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, + // Creates batch size + batch_size: usize, +} + +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 +// `ExhaustedStreamBatch` once stream batches are exhausted. +// 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 matched/unmatched 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, + ) -> Self { + let streamed_schema = streamed.schema(); + Self { + schema, + on_streamed, + join_type, + operator, + streamed_schema, + streamed, + buffered_side, + state, + sort_option, + join_metrics, + batch_process_state: BatchProcessState::new(), + batch_size, + } + } + + 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 => { + self.state = PiecewiseMergeJoinStreamState::ExhaustedStreamSide; + } + 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)?; + + 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()?; + + 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, + self.batch_size, + )?; + + if self.batch_process_state.continue_process { + return Ok(StatefulStreamResult::Ready(Some(batch))); + } + + self.state = PiecewiseMergeJoinStreamState::FetchStreamBatch; + 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)); + } + + let timer = self.join_metrics.join_time.timer(); + + let buffered_data = + Arc::clone(&self.buffered_side.try_as_ready().unwrap().buffered_data); + + // Check if the same batch needs to be checked for values again + if let Some(start_idx) = self.batch_process_state.process_rest { + if let Some(buffered_indices) = &self.batch_process_state.buffered_indices { + let remaining = buffered_indices.len() - start_idx; + + // Branch into this and return value if there are more rows to deal with + if remaining > self.batch_size { + let buffered_batch = buffered_data.batch(); + let empty_stream_batch = + RecordBatch::new_empty(Arc::clone(&self.streamed_schema)); + + let buffered_chunk_ref = + buffered_indices.slice(start_idx, self.batch_size); + let new_buffered_indices = buffered_chunk_ref + .as_any() + .downcast_ref::<UInt64Array>() + .expect("downcast to UInt64Array after slice"); + + let streamed_indices: UInt32Array = + (0..new_buffered_indices.len() as u32).collect(); + + let batch = build_matched_indices( + Arc::clone(&self.schema), + &empty_stream_batch, + buffered_batch, + streamed_indices, + new_buffered_indices.clone(), + )?; + + self.batch_process_state + .set_process_rest(Some(start_idx + self.batch_size)); + self.batch_process_state.continue_process = true; + + return Ok(StatefulStreamResult::Ready(Some(batch))); + } + + let buffered_batch = buffered_data.batch(); + let empty_stream_batch = + RecordBatch::new_empty(Arc::clone(&self.streamed_schema)); + + let buffered_chunk_ref = buffered_indices.slice(start_idx, remaining); + let new_buffered_indices = buffered_chunk_ref + .as_any() + .downcast_ref::<UInt64Array>() + .expect("downcast to UInt64Array after slice"); + + let streamed_indices: UInt32Array = + (0..new_buffered_indices.len() as u32).collect(); + + let batch = build_matched_indices( + Arc::clone(&self.schema), + &empty_stream_batch, + buffered_batch, + streamed_indices, + new_buffered_indices.clone(), + )?; + + self.batch_process_state.reset(); + + timer.done(); + self.join_metrics.output_batches.add(1); + self.state = PiecewiseMergeJoinStreamState::Completed; + + return Ok(StatefulStreamResult::Ready(Some(batch))); + } + + return exec_err!("Batch process state should hold buffered indices"); + } + + let (buffered_indices, streamed_indices) = get_final_indices_from_shared_bitmap( + &buffered_data.visited_indices_bitmap, + self.join_type, + true, + ); + + // If the output indices is larger than the limit for the incremental batching then + // proceed to outputting all matches up to that index, return batch, and the matching + // will start next on the updated index (`process_rest`) + if buffered_indices.len() > self.batch_size { + let buffered_batch = buffered_data.batch(); + let empty_stream_batch = + RecordBatch::new_empty(Arc::clone(&self.streamed_schema)); + + let indices_chunk_ref = buffered_indices + .slice(self.batch_process_state.start_idx, self.batch_size); + + let indices_chunk = indices_chunk_ref + .as_any() + .downcast_ref::<UInt64Array>() + .expect("downcast to UInt64Array after slice"); + + let batch = build_matched_indices( + Arc::clone(&self.schema), + &empty_stream_batch, + buffered_batch, + streamed_indices, + indices_chunk.clone(), + )?; + + self.batch_process_state.buffered_indices = Some(buffered_indices); + self.batch_process_state + .set_process_rest(Some(self.batch_size)); + self.batch_process_state.continue_process = true; + + return Ok(StatefulStreamResult::Ready(Some(batch))); + } + + let buffered_batch = buffered_data.batch(); + let empty_stream_batch = + RecordBatch::new_empty(Arc::clone(&self.streamed_schema)); + + let batch = build_matched_indices( + Arc::clone(&self.schema), + &empty_stream_batch, + buffered_batch, + streamed_indices, + buffered_indices, + )?; + + timer.done(); + self.join_metrics.output_batches.add(1); + self.state = PiecewiseMergeJoinStreamState::Completed; + + Ok(StatefulStreamResult::Ready(Some(batch))) + } +} + +// Holds all information for processing incremental output Review Comment: No i was planning on doing this in a follow up, should this instead be tackled in this pull request? -- 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