-
Notifications
You must be signed in to change notification settings - Fork 4k
ARROW-12432: [Rust] [DataFusion] Add metrics to SortExec #10078
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
Closed
Closed
Changes from all commits
Commits
Show all changes
6 commits
Select commit
Hold shift + click to select a range
db18f79
Add metrics to SortExec
andygrove 0e375f2
Improve ergonomics and add test for SortExec
andygrove fb05d02
pin flatbuffers
andygrove d5dd2bb
Merge remote-tracking branch 'apache/master' into sortexec-metrics
andygrove 471d07f
removed pinned flatbuffer version
andygrove bf49dda
Address feedback
andygrove 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 hidden or 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 hidden or 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 hidden or 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 |
|---|---|---|
|
|
@@ -19,11 +19,14 @@ | |
|
|
||
| use std::any::Any; | ||
| use std::pin::Pin; | ||
| use std::sync::Arc; | ||
| use std::sync::{Arc, Mutex}; | ||
| use std::task::{Context, Poll}; | ||
| use std::time::Instant; | ||
|
|
||
| use async_trait::async_trait; | ||
| use futures::stream::Stream; | ||
| use futures::Future; | ||
| use hashbrown::HashMap; | ||
|
|
||
| use pin_project_lite::pin_project; | ||
|
|
||
|
|
@@ -37,9 +40,9 @@ use arrow::{array::ArrayRef, error::ArrowError}; | |
| use super::{RecordBatchStream, SendableRecordBatchStream}; | ||
| use crate::error::{DataFusionError, Result}; | ||
| use crate::physical_plan::expressions::PhysicalSortExpr; | ||
| use crate::physical_plan::{common, Distribution, ExecutionPlan, Partitioning}; | ||
|
|
||
| use async_trait::async_trait; | ||
| use crate::physical_plan::{ | ||
| common, Distribution, ExecutionPlan, Partitioning, SQLMetric, | ||
| }; | ||
|
|
||
| /// Sort execution plan | ||
| #[derive(Debug)] | ||
|
|
@@ -48,6 +51,10 @@ pub struct SortExec { | |
| input: Arc<dyn ExecutionPlan>, | ||
| /// Sort expressions | ||
| expr: Vec<PhysicalSortExpr>, | ||
| /// Output rows | ||
| output_rows: Arc<Mutex<SQLMetric>>, | ||
| /// Time to sort batches | ||
| sort_time_nanos: Arc<Mutex<SQLMetric>>, | ||
| } | ||
|
|
||
| impl SortExec { | ||
|
|
@@ -56,7 +63,12 @@ impl SortExec { | |
| expr: Vec<PhysicalSortExpr>, | ||
| input: Arc<dyn ExecutionPlan>, | ||
| ) -> Result<Self> { | ||
| Ok(Self { expr, input }) | ||
| Ok(Self { | ||
| expr, | ||
| input, | ||
| output_rows: SQLMetric::counter("outputRows"), | ||
| sort_time_nanos: SQLMetric::time_nanos("sortTime"), | ||
| }) | ||
| } | ||
|
|
||
| /// Input schema | ||
|
|
@@ -125,7 +137,25 @@ impl ExecutionPlan for SortExec { | |
| } | ||
| let input = self.input.execute(0).await?; | ||
|
|
||
| Ok(Box::pin(SortStream::new(input, self.expr.clone()))) | ||
| Ok(Box::pin(SortStream::new( | ||
| input, | ||
| self.expr.clone(), | ||
| self.output_rows.clone(), | ||
| self.sort_time_nanos.clone(), | ||
| ))) | ||
| } | ||
|
|
||
| fn metrics(&self) -> HashMap<String, SQLMetric> { | ||
| let mut metrics = HashMap::new(); | ||
| metrics.insert( | ||
| "outputRows".to_owned(), | ||
| self.output_rows.lock().unwrap().clone(), | ||
| ); | ||
| metrics.insert( | ||
| "sortTime".to_owned(), | ||
| self.sort_time_nanos.lock().unwrap().clone(), | ||
| ); | ||
| metrics | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -194,11 +224,17 @@ pin_project! { | |
| output: futures::channel::oneshot::Receiver<ArrowResult<Option<RecordBatch>>>, | ||
| finished: bool, | ||
| schema: SchemaRef, | ||
| output_rows: Arc<Mutex<SQLMetric>>, | ||
| } | ||
| } | ||
|
|
||
| impl SortStream { | ||
| fn new(input: SendableRecordBatchStream, expr: Vec<PhysicalSortExpr>) -> Self { | ||
| fn new( | ||
| input: SendableRecordBatchStream, | ||
| expr: Vec<PhysicalSortExpr>, | ||
| output_rows: Arc<Mutex<SQLMetric>>, | ||
| sort_time: Arc<Mutex<SQLMetric>>, | ||
| ) -> Self { | ||
| let (tx, rx) = futures::channel::oneshot::channel(); | ||
|
|
||
| let schema = input.schema(); | ||
|
|
@@ -207,7 +243,13 @@ impl SortStream { | |
| let sorted_batch = common::collect(input) | ||
| .await | ||
| .map_err(DataFusionError::into_arrow_external_error) | ||
| .and_then(move |batches| sort_batches(&batches, &schema, &expr)); | ||
| .and_then(move |batches| { | ||
| let now = Instant::now(); | ||
| let result = sort_batches(&batches, &schema, &expr); | ||
| let mut sort_time = sort_time.lock().unwrap(); | ||
| sort_time.add(now.elapsed().as_nanos() as usize); | ||
| result | ||
| }); | ||
|
|
||
| tx.send(sorted_batch) | ||
| }); | ||
|
|
@@ -216,6 +258,7 @@ impl SortStream { | |
| output: rx, | ||
| finished: false, | ||
| schema, | ||
| output_rows, | ||
| } | ||
| } | ||
| } | ||
|
|
@@ -224,6 +267,8 @@ impl Stream for SortStream { | |
| type Item = ArrowResult<RecordBatch>; | ||
|
|
||
| fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> { | ||
| let output_rows = self.output_rows.clone(); | ||
|
|
||
| if self.finished { | ||
| return Poll::Ready(None); | ||
| } | ||
|
|
@@ -241,6 +286,12 @@ impl Stream for SortStream { | |
| Err(e) => Some(Err(ArrowError::ExternalError(Box::new(e)))), // error receiving | ||
| Ok(result) => result.transpose(), | ||
| }; | ||
|
|
||
| if let Some(Ok(batch)) = &result { | ||
| let mut output_rows = output_rows.lock().unwrap(); | ||
| output_rows.add(batch.num_rows()); | ||
| } | ||
|
|
||
| Poll::Ready(result) | ||
| } | ||
| Poll::Pending => Poll::Pending, | ||
|
|
@@ -379,7 +430,9 @@ mod tests { | |
| assert_eq!(DataType::Float32, *sort_exec.schema().field(0).data_type()); | ||
| assert_eq!(DataType::Float64, *sort_exec.schema().field(1).data_type()); | ||
|
|
||
| let result: Vec<RecordBatch> = collect(sort_exec).await?; | ||
| let result: Vec<RecordBatch> = collect(sort_exec.clone()).await?; | ||
| assert!(sort_exec.metrics().get("sortTime").unwrap().value > 0); | ||
| assert_eq!(sort_exec.metrics().get("outputRows").unwrap().value, 8); | ||
| assert_eq!(result.len(), 1); | ||
|
Contributor
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. Maybe also assert that the time counter was greater than zero? |
||
|
|
||
| let columns = result[0].columns(); | ||
|
|
||
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.
Given Rust's focus on compile time type checking, what would you think about using typed counters rather than
Stringkeys?So make the code look something like:
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.
Does this imply adding an enum for the metrics? This might limit extensibility for users that want to add custom metrics.