-
Notifications
You must be signed in to change notification settings - Fork 1.4k
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
Don't optimize AnalyzeExec (#6379) (try 2) #6494
Changes from all commits
b7e84dd
991ab18
805f2f3
aadf685
543b94e
5521a70
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -29,9 +29,11 @@ use crate::{ | |
}; | ||
use arrow::{array::StringBuilder, datatypes::SchemaRef, record_batch::RecordBatch}; | ||
use futures::StreamExt; | ||
use tokio::task::JoinSet; | ||
|
||
use super::expressions::PhysicalSortExpr; | ||
use super::{stream::RecordBatchReceiverStream, Distribution, SendableRecordBatchStream}; | ||
use super::stream::RecordBatchStreamAdapter; | ||
use super::{Distribution, SendableRecordBatchStream}; | ||
use crate::execution::context::TaskContext; | ||
|
||
/// `EXPLAIN ANALYZE` execution plan operator. This operator runs its input, | ||
|
@@ -71,23 +73,18 @@ impl ExecutionPlan for AnalyzeExec { | |
vec![self.input.clone()] | ||
} | ||
|
||
/// Specifies we want the input as a single stream | ||
/// AnalyzeExec is handled specially so this value is ignored | ||
fn required_input_distribution(&self) -> Vec<Distribution> { | ||
vec![Distribution::SinglePartition] | ||
vec![] | ||
} | ||
|
||
/// Specifies whether this plan generates an infinite stream of records. | ||
/// If the plan does not support pipelining, but its input(s) are | ||
/// infinite, returns an error to indicate this. | ||
fn unbounded_output(&self, children: &[bool]) -> Result<bool> { | ||
if children[0] { | ||
Err(DataFusionError::Plan( | ||
"Analyze Error: Analysis is not supported for unbounded inputs" | ||
.to_string(), | ||
)) | ||
} else { | ||
Ok(false) | ||
} | ||
fn unbounded_output(&self, _children: &[bool]) -> Result<bool> { | ||
Err(DataFusionError::Internal( | ||
"Optimization not supported for ANALYZE".to_string(), | ||
)) | ||
} | ||
|
||
/// Get the output partitioning of this plan | ||
|
@@ -121,96 +118,62 @@ impl ExecutionPlan for AnalyzeExec { | |
))); | ||
} | ||
|
||
// should be ensured by `SinglePartition` above | ||
let input_partitions = self.input.output_partitioning().partition_count(); | ||
if input_partitions != 1 { | ||
return Err(DataFusionError::Internal(format!( | ||
"AnalyzeExec invalid number of input partitions. Expected 1, got {input_partitions}" | ||
))); | ||
// Gather futures that will run each input partition in | ||
// parallel (on a separate tokio task) using a JoinSet to | ||
// cancel outstanding futures on drop | ||
let mut set = JoinSet::new(); | ||
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. This uses the cool 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. this logic was just extracted into its own function |
||
let num_input_partitions = self.input.output_partitioning().partition_count(); | ||
|
||
for input_partition in 0..num_input_partitions { | ||
let input_stream = self.input.execute(input_partition, context.clone()); | ||
|
||
set.spawn(async move { | ||
let mut total_rows = 0; | ||
let mut input_stream = input_stream?; | ||
while let Some(batch) = input_stream.next().await { | ||
let batch = batch?; | ||
total_rows += batch.num_rows(); | ||
} | ||
Ok(total_rows) as Result<usize> | ||
}); | ||
} | ||
|
||
let (tx, rx) = tokio::sync::mpsc::channel(input_partitions); | ||
|
||
let start = Instant::now(); | ||
let captured_input = self.input.clone(); | ||
let mut input_stream = captured_input.execute(0, context)?; | ||
let captured_schema = self.schema.clone(); | ||
let verbose = self.verbose; | ||
|
||
// Task reads batches the input and when complete produce a | ||
// RecordBatch with a report that is written to `tx` when done | ||
let join_handle = tokio::task::spawn(async move { | ||
let start = Instant::now(); | ||
// future that gathers the results from all the tasks in the | ||
// JoinSet that computes the overall row count and final | ||
// record batch | ||
let output = async move { | ||
let mut total_rows = 0; | ||
|
||
// Note the code below ignores errors sending on tx. An | ||
// error sending means the plan is being torn down and | ||
// nothing is left that will handle the error (aka no one | ||
// will hear us scream) | ||
while let Some(b) = input_stream.next().await { | ||
match b { | ||
Ok(batch) => { | ||
total_rows += batch.num_rows(); | ||
} | ||
b @ Err(_) => { | ||
// try and pass on errors from input | ||
if tx.send(b).await.is_err() { | ||
// receiver hung up, stop executing (no | ||
// one will look at any further results we | ||
// send) | ||
return; | ||
} | ||
while let Some(res) = set.join_next().await { | ||
// translate join errors (aka task panic's) into ExecutionErrors | ||
match res { | ||
Ok(row_count) => total_rows += row_count?, | ||
Err(e) => { | ||
return Err(DataFusionError::Execution(format!( | ||
"Join error in AnalyzeExec: {e}" | ||
))) | ||
} | ||
} | ||
} | ||
let end = Instant::now(); | ||
|
||
let mut type_builder = StringBuilder::with_capacity(1, 1024); | ||
let mut plan_builder = StringBuilder::with_capacity(1, 1024); | ||
|
||
// TODO use some sort of enum rather than strings? | ||
type_builder.append_value("Plan with Metrics"); | ||
|
||
let annotated_plan = | ||
DisplayableExecutionPlan::with_metrics(captured_input.as_ref()) | ||
.indent() | ||
.to_string(); | ||
plan_builder.append_value(annotated_plan); | ||
|
||
// Verbose output | ||
// TODO make this more sophisticated | ||
if verbose { | ||
type_builder.append_value("Plan with Full Metrics"); | ||
|
||
let annotated_plan = | ||
DisplayableExecutionPlan::with_full_metrics(captured_input.as_ref()) | ||
.indent() | ||
.to_string(); | ||
plan_builder.append_value(annotated_plan); | ||
|
||
type_builder.append_value("Output Rows"); | ||
plan_builder.append_value(total_rows.to_string()); | ||
|
||
type_builder.append_value("Duration"); | ||
plan_builder.append_value(format!("{:?}", end - start)); | ||
} | ||
|
||
let maybe_batch = RecordBatch::try_new( | ||
let duration = Instant::now() - start; | ||
create_output_batch( | ||
verbose, | ||
total_rows, | ||
duration, | ||
captured_input, | ||
captured_schema, | ||
vec![ | ||
Arc::new(type_builder.finish()), | ||
Arc::new(plan_builder.finish()), | ||
], | ||
) | ||
.map_err(Into::into); | ||
// again ignore error | ||
tx.send(maybe_batch).await.ok(); | ||
}); | ||
|
||
Ok(RecordBatchReceiverStream::create( | ||
&self.schema, | ||
rx, | ||
join_handle, | ||
)) | ||
}; | ||
|
||
Ok(Box::pin(RecordBatchStreamAdapter::new( | ||
self.schema.clone(), | ||
futures::stream::once(output), | ||
))) | ||
} | ||
|
||
fn fmt_as( | ||
|
@@ -231,6 +194,52 @@ impl ExecutionPlan for AnalyzeExec { | |
} | ||
} | ||
|
||
/// Creates the ouput of AnalyzeExec as a RecordBatch | ||
fn create_output_batch( | ||
verbose: bool, | ||
total_rows: usize, | ||
duration: std::time::Duration, | ||
input: Arc<dyn ExecutionPlan>, | ||
schema: SchemaRef, | ||
) -> Result<RecordBatch> { | ||
let mut type_builder = StringBuilder::with_capacity(1, 1024); | ||
let mut plan_builder = StringBuilder::with_capacity(1, 1024); | ||
|
||
// TODO use some sort of enum rather than strings? | ||
type_builder.append_value("Plan with Metrics"); | ||
|
||
let annotated_plan = DisplayableExecutionPlan::with_metrics(input.as_ref()) | ||
.indent() | ||
.to_string(); | ||
plan_builder.append_value(annotated_plan); | ||
|
||
// Verbose output | ||
// TODO make this more sophisticated | ||
if verbose { | ||
type_builder.append_value("Plan with Full Metrics"); | ||
|
||
let annotated_plan = DisplayableExecutionPlan::with_full_metrics(input.as_ref()) | ||
.indent() | ||
.to_string(); | ||
plan_builder.append_value(annotated_plan); | ||
|
||
type_builder.append_value("Output Rows"); | ||
plan_builder.append_value(total_rows.to_string()); | ||
|
||
type_builder.append_value("Duration"); | ||
plan_builder.append_value(format!("{:?}", duration)); | ||
} | ||
|
||
RecordBatch::try_new( | ||
schema, | ||
vec![ | ||
Arc::new(type_builder.finish()), | ||
Arc::new(plan_builder.finish()), | ||
], | ||
) | ||
.map_err(DataFusionError::from) | ||
} | ||
|
||
#[cfg(test)] | ||
mod tests { | ||
use arrow::datatypes::{DataType, Field, Schema}; | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -687,13 +687,31 @@ async fn csv_explain_analyze() { | |
// Only test basic plumbing and try to avoid having to change too | ||
// many things. explain_analyze_baseline_metrics covers the values | ||
// in greater depth | ||
let needle = "CoalescePartitionsExec, metrics=[output_rows=5, elapsed_compute="; | ||
let needle = "AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[COUNT(UInt8(1))], metrics=[output_rows=5"; | ||
assert_contains!(&formatted, needle); | ||
|
||
let verbose_needle = "Output Rows"; | ||
assert_not_contains!(formatted, verbose_needle); | ||
} | ||
|
||
#[tokio::test] | ||
#[cfg_attr(tarpaulin, ignore)] | ||
async fn csv_explain_analyze_order_by() { | ||
let ctx = SessionContext::new(); | ||
register_aggregate_csv_by_sql(&ctx).await; | ||
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. here is the new test |
||
let sql = "EXPLAIN ANALYZE SELECT c1 FROM aggregate_test_100 order by c1"; | ||
let actual = execute_to_batches(&ctx, sql).await; | ||
let formatted = arrow::util::pretty::pretty_format_batches(&actual) | ||
.unwrap() | ||
.to_string(); | ||
|
||
// Ensure that the ordering is not optimized away from the plan | ||
// https://github.com/apache/arrow-datafusion/issues/6379 | ||
let needle = | ||
"SortExec: expr=[c1@0 ASC NULLS LAST], metrics=[output_rows=100, elapsed_compute"; | ||
assert_contains!(&formatted, needle); | ||
} | ||
|
||
#[tokio::test] | ||
#[cfg_attr(tarpaulin, ignore)] | ||
async fn parquet_explain_analyze() { | ||
|
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.
it works now!