-
Notifications
You must be signed in to change notification settings - Fork 1.3k
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 reservation & metrics for cross join #5339
Merged
Merged
Changes from 1 commit
Commits
Show all changes
3 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -26,6 +26,9 @@ use arrow::datatypes::{Schema, SchemaRef}; | |
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::metrics::{ExecutionPlanMetricsSet, MetricsSet}; | ||
use crate::physical_plan::{ | ||
coalesce_batches::concat_batches, coalesce_partitions::CoalescePartitionsExec, | ||
ColumnStatistics, DisplayFormatType, Distribution, EquivalenceProperties, | ||
|
@@ -35,12 +38,11 @@ use crate::physical_plan::{ | |
use crate::{error::Result, scalar::ScalarValue}; | ||
use async_trait::async_trait; | ||
use datafusion_common::DataFusionError; | ||
use log::debug; | ||
use std::time::Instant; | ||
use parking_lot::Mutex; | ||
|
||
use super::utils::{ | ||
adjust_right_output_partitioning, cross_join_equivalence_properties, OnceAsync, | ||
OnceFut, | ||
adjust_right_output_partitioning, cross_join_equivalence_properties, | ||
BuildProbeJoinMetrics, OnceAsync, OnceFut, | ||
}; | ||
|
||
/// Data of the left side | ||
|
@@ -58,6 +60,8 @@ pub struct CrossJoinExec { | |
schema: SchemaRef, | ||
/// Build-side data | ||
left_fut: OnceAsync<JoinLeftData>, | ||
/// Execution plan metrics | ||
metrics: ExecutionPlanMetricsSet, | ||
} | ||
|
||
impl CrossJoinExec { | ||
|
@@ -79,6 +83,7 @@ impl CrossJoinExec { | |
right, | ||
schema, | ||
left_fut: Default::default(), | ||
metrics: ExecutionPlanMetricsSet::default(), | ||
} | ||
} | ||
|
||
|
@@ -97,9 +102,9 @@ impl CrossJoinExec { | |
async fn load_left_input( | ||
left: Arc<dyn ExecutionPlan>, | ||
context: Arc<TaskContext>, | ||
metrics: BuildProbeJoinMetrics, | ||
reservation: SharedMemoryReservation, | ||
) -> Result<JoinLeftData> { | ||
let start = Instant::now(); | ||
|
||
// merge all left parts into a single stream | ||
let merge = { | ||
if left.output_partitioning().partition_count() != 1 { | ||
|
@@ -111,22 +116,28 @@ async fn load_left_input( | |
let stream = merge.execute(0, context)?; | ||
|
||
// Load all batches and count the rows | ||
let (batches, num_rows) = stream | ||
.try_fold((Vec::new(), 0usize), |mut acc, batch| async { | ||
acc.1 += batch.num_rows(); | ||
acc.0.push(batch); | ||
Ok(acc) | ||
}) | ||
let (batches, num_rows, _, _) = stream | ||
.try_fold( | ||
(Vec::new(), 0usize, metrics, reservation), | ||
|mut acc, batch| async { | ||
let batch_size = batch.get_array_memory_size(); | ||
// Reserve memory for incoming batch | ||
acc.3.lock().try_grow(batch_size)?; | ||
// Update metrics | ||
acc.2.build_mem_used.add(batch_size); | ||
acc.2.build_input_batches.add(1); | ||
acc.2.build_input_rows.add(batch.num_rows()); | ||
// Update rowcount | ||
acc.1 += batch.num_rows(); | ||
// Push batch to output | ||
acc.0.push(batch); | ||
Ok(acc) | ||
}, | ||
) | ||
.await?; | ||
|
||
let merged_batch = concat_batches(&left.schema(), &batches, num_rows)?; | ||
|
||
debug!( | ||
"Built build-side of cross join containing {} rows in {} ms", | ||
num_rows, | ||
start.elapsed().as_millis() | ||
); | ||
|
||
Ok(merged_batch) | ||
} | ||
|
||
|
@@ -143,6 +154,10 @@ impl ExecutionPlan for CrossJoinExec { | |
vec![self.left.clone(), self.right.clone()] | ||
} | ||
|
||
fn metrics(&self) -> Option<MetricsSet> { | ||
Some(self.metrics.clone_inner()) | ||
} | ||
|
||
/// Specifies whether this plan generates an infinite stream of records. | ||
/// If the plan does not support pipelining, but it its input(s) are | ||
/// infinite, returns an error to indicate this. | ||
|
@@ -205,21 +220,29 @@ impl ExecutionPlan for CrossJoinExec { | |
) -> Result<SendableRecordBatchStream> { | ||
let stream = self.right.execute(partition, context.clone())?; | ||
|
||
let left_fut = self | ||
.left_fut | ||
.once(|| load_left_input(self.left.clone(), context)); | ||
let join_metrics = BuildProbeJoinMetrics::new(partition, &self.metrics); | ||
let reservation = Arc::new(Mutex::new( | ||
MemoryConsumer::new(format!("CrossJoinStream[{partition}]")) | ||
.register(context.memory_pool()), | ||
)); | ||
|
||
let left_fut = self.left_fut.once(|| { | ||
load_left_input( | ||
self.left.clone(), | ||
context, | ||
join_metrics.clone(), | ||
reservation.clone(), | ||
) | ||
}); | ||
|
||
Ok(Box::pin(CrossJoinStream { | ||
schema: self.schema.clone(), | ||
left_fut, | ||
right: stream, | ||
right_batch: Arc::new(parking_lot::Mutex::new(None)), | ||
left_index: 0, | ||
num_input_batches: 0, | ||
num_input_rows: 0, | ||
num_output_batches: 0, | ||
num_output_rows: 0, | ||
join_time: 0, | ||
join_metrics, | ||
reservation, | ||
})) | ||
} | ||
|
||
|
@@ -321,16 +344,10 @@ struct CrossJoinStream { | |
left_index: usize, | ||
/// Current batch being processed from the right side | ||
right_batch: Arc<parking_lot::Mutex<Option<RecordBatch>>>, | ||
/// number of input batches | ||
num_input_batches: usize, | ||
/// number of input rows | ||
num_input_rows: usize, | ||
/// number of batches produced | ||
num_output_batches: usize, | ||
/// number of rows produced | ||
num_output_rows: usize, | ||
/// total time for joining probe-side batches to the build-side batches | ||
join_time: usize, | ||
/// join execution metrics | ||
join_metrics: BuildProbeJoinMetrics, | ||
/// memory reservation | ||
reservation: SharedMemoryReservation, | ||
} | ||
|
||
impl RecordBatchStream for CrossJoinStream { | ||
|
@@ -385,28 +402,30 @@ impl CrossJoinStream { | |
&mut self, | ||
cx: &mut std::task::Context<'_>, | ||
) -> std::task::Poll<Option<Result<RecordBatch>>> { | ||
let build_timer = self.join_metrics.build_time.timer(); | ||
let left_data = match ready!(self.left_fut.get(cx)) { | ||
Ok(left_data) => left_data, | ||
Err(e) => return Poll::Ready(Some(Err(e))), | ||
}; | ||
build_timer.done(); | ||
|
||
if left_data.num_rows() == 0 { | ||
return Poll::Ready(None); | ||
} | ||
|
||
if self.left_index > 0 && self.left_index < left_data.num_rows() { | ||
let start = Instant::now(); | ||
let join_timer = self.join_metrics.join_time.timer(); | ||
let right_batch = { | ||
let right_batch = self.right_batch.lock(); | ||
right_batch.clone().unwrap() | ||
}; | ||
let result = | ||
build_batch(self.left_index, &right_batch, left_data, &self.schema); | ||
self.num_input_rows += right_batch.num_rows(); | ||
self.join_metrics.input_rows.add(right_batch.num_rows()); | ||
if let Ok(ref batch) = result { | ||
self.join_time += start.elapsed().as_millis() as usize; | ||
self.num_output_batches += 1; | ||
self.num_output_rows += batch.num_rows(); | ||
join_timer.done(); | ||
self.join_metrics.output_batches.add(1); | ||
self.join_metrics.output_rows.add(batch.num_rows()); | ||
} | ||
self.left_index += 1; | ||
return Poll::Ready(Some(result)); | ||
|
@@ -416,15 +435,15 @@ impl CrossJoinStream { | |
.poll_next_unpin(cx) | ||
.map(|maybe_batch| match maybe_batch { | ||
Some(Ok(batch)) => { | ||
let start = Instant::now(); | ||
let join_timer = self.join_metrics.join_time.timer(); | ||
let result = | ||
build_batch(self.left_index, &batch, left_data, &self.schema); | ||
self.num_input_batches += 1; | ||
self.num_input_rows += batch.num_rows(); | ||
self.join_metrics.input_batches.add(1); | ||
self.join_metrics.input_rows.add(batch.num_rows()); | ||
if let Ok(ref batch) = result { | ||
self.join_time += start.elapsed().as_millis() as usize; | ||
self.num_output_batches += 1; | ||
self.num_output_rows += batch.num_rows(); | ||
join_timer.done(); | ||
self.join_metrics.output_batches.add(1); | ||
self.join_metrics.output_rows.add(batch.num_rows()); | ||
} | ||
self.left_index = 1; | ||
|
||
|
@@ -434,15 +453,7 @@ impl CrossJoinStream { | |
Some(result) | ||
} | ||
other => { | ||
debug!( | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think it is a nice improvement that the metrics are now included in things like |
||
"Processed {} probe-side input batches containing {} rows and \ | ||
produced {} output batches containing {} rows in {} ms", | ||
self.num_input_batches, | ||
self.num_input_rows, | ||
self.num_output_batches, | ||
self.num_output_rows, | ||
self.join_time | ||
); | ||
self.reservation.lock().free(); | ||
other | ||
} | ||
}) | ||
|
@@ -452,6 +463,25 @@ impl CrossJoinStream { | |
#[cfg(test)] | ||
mod tests { | ||
use super::*; | ||
use crate::assert_batches_sorted_eq; | ||
use crate::execution::runtime_env::{RuntimeConfig, RuntimeEnv}; | ||
use crate::physical_plan::common; | ||
use crate::prelude::{SessionConfig, SessionContext}; | ||
use crate::test::{build_table_scan_i32, columns}; | ||
|
||
async fn join_collect( | ||
left: Arc<dyn ExecutionPlan>, | ||
right: Arc<dyn ExecutionPlan>, | ||
context: Arc<TaskContext>, | ||
) -> Result<(Vec<String>, Vec<RecordBatch>)> { | ||
let join = CrossJoinExec::new(left, right); | ||
let columns_header = columns(&join.schema()); | ||
|
||
let stream = join.execute(0, context)?; | ||
let batches = common::collect(stream).await?; | ||
|
||
Ok((columns_header, batches)) | ||
} | ||
|
||
#[tokio::test] | ||
async fn test_stats_cartesian_product() { | ||
|
@@ -589,4 +619,72 @@ mod tests { | |
|
||
assert_eq!(result, expected); | ||
} | ||
|
||
#[tokio::test] | ||
async fn test_join() -> Result<()> { | ||
let session_ctx = SessionContext::new(); | ||
let task_ctx = session_ctx.task_ctx(); | ||
|
||
let left = build_table_scan_i32( | ||
("a1", &vec![1, 2, 3]), | ||
("b1", &vec![4, 5, 6]), | ||
("c1", &vec![7, 8, 9]), | ||
); | ||
let right = build_table_scan_i32( | ||
("a2", &vec![10, 11]), | ||
("b2", &vec![12, 13]), | ||
("c2", &vec![14, 15]), | ||
); | ||
|
||
let (columns, batches) = join_collect(left, right, task_ctx).await?; | ||
|
||
assert_eq!(columns, vec!["a1", "b1", "c1", "a2", "b2", "c2"]); | ||
let expected = vec![ | ||
"+----+----+----+----+----+----+", | ||
"| a1 | b1 | c1 | a2 | b2 | c2 |", | ||
"+----+----+----+----+----+----+", | ||
"| 1 | 4 | 7 | 10 | 12 | 14 |", | ||
"| 1 | 4 | 7 | 11 | 13 | 15 |", | ||
"| 2 | 5 | 8 | 10 | 12 | 14 |", | ||
"| 2 | 5 | 8 | 11 | 13 | 15 |", | ||
"| 3 | 6 | 9 | 10 | 12 | 14 |", | ||
"| 3 | 6 | 9 | 11 | 13 | 15 |", | ||
"+----+----+----+----+----+----+", | ||
]; | ||
|
||
assert_batches_sorted_eq!(expected, &batches); | ||
|
||
Ok(()) | ||
} | ||
|
||
#[tokio::test] | ||
async fn test_overallocation() -> Result<()> { | ||
let runtime_config = RuntimeConfig::new().with_memory_limit(100, 1.0); | ||
let runtime = Arc::new(RuntimeEnv::new(runtime_config)?); | ||
let session_ctx = | ||
SessionContext::with_config_rt(SessionConfig::default(), runtime); | ||
let task_ctx = session_ctx.task_ctx(); | ||
|
||
let left = build_table_scan_i32( | ||
("a1", &vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 0]), | ||
("b1", &vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 0]), | ||
("c1", &vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 0]), | ||
); | ||
let right = build_table_scan_i32( | ||
("a2", &vec![10, 11]), | ||
("b2", &vec![12, 13]), | ||
("c2", &vec![14, 15]), | ||
); | ||
|
||
let err = join_collect(left, right, task_ctx).await.unwrap_err(); | ||
|
||
assert_eq!( | ||
err.to_string(), | ||
"External error: Resources exhausted: Failed to allocate \ | ||
additional 744 bytes for CrossJoinStream[0] with 0 bytes \ | ||
already allocated - maximum available is 100" | ||
); | ||
|
||
Ok(()) | ||
} | ||
} |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
❤️