Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Memory limited hash join #5490

Merged
merged 1 commit into from
Mar 9, 2023
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
6 changes: 6 additions & 0 deletions datafusion/core/src/physical_plan/common.rs
Original file line number Diff line number Diff line change
Expand Up @@ -39,8 +39,14 @@ use std::task::{Context, Poll};
use tokio::sync::mpsc;
use tokio::task::JoinHandle;

/// [`MemoryReservation`] used across query execution streams
pub(crate) type SharedMemoryReservation = Arc<Mutex<MemoryReservation>>;

/// [`MemoryReservation`] used at query operator level
/// `Option` wrapper allows to initialize empty reservation in operator constructor,
/// and set it to actual reservation at stream level.
pub(crate) type OperatorMemoryReservation = Arc<Mutex<Option<SharedMemoryReservation>>>;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I wonder if the double layer of Mutex is necessary (Mutex over an Option of a Mutex)

Update after reading the code I see why it is necessary.

Perhaps it would make the code clearer (as a follow on PR, perhaps) if we had a proper SharedMemoryReservation that could be cloned and handled mutability internally. 🤔

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Unfortunately I haven't found better solution yet, but I'll check if there is better way to wrap MemoryReservation into something allowing attempts of creation (or initialization) reservation from multiple streams.

Another solution, probably, is to allow to allow creation of MemoryReservation without providing context (in HashJoinExec::new), and registering it in MemoryPool later (in HashJoinExec.execute, when we have context available).

Copy link
Contributor Author

@korowa korowa Mar 9, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

So, if there are no objections, I'd prefer to figure out proper way to handle operator-level reservations as a separate follow-up PR, or during adding memory limitations to one of remaining joins (at least NL will have the same issue 100%)


/// Stream of record batches
pub struct SizedRecordBatchStream {
schema: SchemaRef,
Expand Down
36 changes: 23 additions & 13 deletions datafusion/core/src/physical_plan/joins/cross_join.rs
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@ use arrow::record_batch::RecordBatch;

use crate::execution::context::TaskContext;
use crate::execution::memory_pool::MemoryConsumer;
use crate::physical_plan::common::SharedMemoryReservation;
use crate::physical_plan::common::{OperatorMemoryReservation, SharedMemoryReservation};
use crate::physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet};
use crate::physical_plan::{
coalesce_batches::concat_batches, coalesce_partitions::CoalescePartitionsExec,
Expand Down Expand Up @@ -60,6 +60,8 @@ pub struct CrossJoinExec {
schema: SchemaRef,
/// Build-side data
left_fut: OnceAsync<JoinLeftData>,
/// Memory reservation for build-side data
reservation: OperatorMemoryReservation,
/// Execution plan metrics
metrics: ExecutionPlanMetricsSet,
}
Expand All @@ -83,6 +85,7 @@ impl CrossJoinExec {
right,
schema,
left_fut: Default::default(),
reservation: Default::default(),
metrics: ExecutionPlanMetricsSet::default(),
}
}
Expand Down Expand Up @@ -221,17 +224,29 @@ impl ExecutionPlan for CrossJoinExec {
let stream = self.right.execute(partition, context.clone())?;

let join_metrics = BuildProbeJoinMetrics::new(partition, &self.metrics);
let reservation = Arc::new(Mutex::new(
MemoryConsumer::new(format!("CrossJoinStream[{partition}]"))
.register(context.memory_pool()),
));

// Initialization of operator-level reservation
{
let mut reservation_lock = self.reservation.lock();
if reservation_lock.is_none() {
*reservation_lock = Some(Arc::new(Mutex::new(
MemoryConsumer::new("CrossJoinExec").register(context.memory_pool()),
)));
};
}

let reservation = self.reservation.lock().clone().ok_or_else(|| {
DataFusionError::Internal(
"Operator-level memory reservation is not initialized".to_string(),
)
})?;

let left_fut = self.left_fut.once(|| {
load_left_input(
self.left.clone(),
context,
join_metrics.clone(),
reservation.clone(),
reservation,
)
});

Expand All @@ -242,7 +257,6 @@ impl ExecutionPlan for CrossJoinExec {
right_batch: Arc::new(parking_lot::Mutex::new(None)),
left_index: 0,
join_metrics,
reservation,
}))
}

Expand Down Expand Up @@ -346,8 +360,6 @@ struct CrossJoinStream {
right_batch: Arc<parking_lot::Mutex<Option<RecordBatch>>>,
/// join execution metrics
join_metrics: BuildProbeJoinMetrics,
/// memory reservation
reservation: SharedMemoryReservation,
}

impl RecordBatchStream for CrossJoinStream {
Expand Down Expand Up @@ -452,10 +464,7 @@ impl CrossJoinStream {

Some(result)
}
other => {
self.reservation.lock().free();
other
}
other => other,
korowa marked this conversation as resolved.
Show resolved Hide resolved
})
}
}
Expand Down Expand Up @@ -683,6 +692,7 @@ mod tests {
err.to_string(),
"External error: Resources exhausted: Failed to allocate additional"
);
assert_contains!(err.to_string(), "CrossJoinExec");
Dandandan marked this conversation as resolved.
Show resolved Hide resolved

Ok(())
}
Expand Down
Loading