Skip to content

Commit

Permalink
Minor: make LeftJoinData into a struct in CrossJoinExec (apache#13227)
Browse files Browse the repository at this point in the history
  • Loading branch information
alamb authored Nov 4, 2024
1 parent d19865e commit 2482ff4
Showing 1 changed file with 17 additions and 5 deletions.
22 changes: 17 additions & 5 deletions datafusion/physical-plan/src/joins/cross_join.rs
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,15 @@ use async_trait::async_trait;
use futures::{ready, Stream, StreamExt, TryStreamExt};

/// Data of the left side
type JoinLeftData = (RecordBatch, MemoryReservation);
#[derive(Debug)]
struct JoinLeftData {
/// Single RecordBatch with all rows from the left side
merged_batch: RecordBatch,
/// Track memory reservation for merged_batch. Relies on drop
/// semantics to release reservation when JoinLeftData is dropped.
#[allow(dead_code)]
reservation: MemoryReservation,
}

#[allow(rustdoc::private_intra_doc_links)]
/// executes partitions in parallel and combines them into a set of
Expand Down Expand Up @@ -185,7 +193,10 @@ async fn load_left_input(

let merged_batch = concat_batches(&left_schema, &batches)?;

Ok((merged_batch, reservation))
Ok(JoinLeftData {
merged_batch,
reservation,
})
}

impl DisplayAs for CrossJoinExec {
Expand Down Expand Up @@ -357,7 +368,7 @@ struct CrossJoinStream<T> {
join_metrics: BuildProbeJoinMetrics,
/// State of the stream
state: CrossJoinStreamState,
/// Left data
/// Left data (copy of the entire buffered left side)
left_data: RecordBatch,
/// Batch transformer
batch_transformer: T,
Expand Down Expand Up @@ -457,16 +468,17 @@ impl<T: BatchTransformer> CrossJoinStream<T> {
cx: &mut std::task::Context<'_>,
) -> Poll<Result<StatefulStreamResult<Option<RecordBatch>>>> {
let build_timer = self.join_metrics.build_time.timer();
let (left_data, _) = match ready!(self.left_fut.get(cx)) {
let left_data = match ready!(self.left_fut.get(cx)) {
Ok(left_data) => left_data,
Err(e) => return Poll::Ready(Err(e)),
};
build_timer.done();

let left_data = left_data.merged_batch.clone();
let result = if left_data.num_rows() == 0 {
StatefulStreamResult::Ready(None)
} else {
self.left_data = left_data.clone();
self.left_data = left_data;
self.state = CrossJoinStreamState::FetchProbeBatch;
StatefulStreamResult::Continue
};
Expand Down

0 comments on commit 2482ff4

Please sign in to comment.