-
Notifications
You must be signed in to change notification settings - Fork 1.9k
Add BaselineMetrics, Timestamp metrics, add for CoalescePartitionsExec, rename output_time -> elapsed_compute
#909
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
Merged
Merged
Changes from all commits
Commits
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 |
|---|---|---|
| @@ -0,0 +1,183 @@ | ||
| // Licensed to the Apache Software Foundation (ASF) under one | ||
| // or more contributor license agreements. See the NOTICE file | ||
| // distributed with this work for additional information | ||
| // regarding copyright ownership. The ASF licenses this file | ||
| // to you under the Apache License, Version 2.0 (the | ||
| // "License"); you may not use this file except in compliance | ||
| // with the License. You may obtain a copy of the License at | ||
| // | ||
| // http://www.apache.org/licenses/LICENSE-2.0 | ||
| // | ||
| // Unless required by applicable law or agreed to in writing, | ||
| // software distributed under the License is distributed on an | ||
| // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
| // KIND, either express or implied. See the License for the | ||
| // specific language governing permissions and limitations | ||
| // under the License. | ||
|
|
||
| //! Metrics common for almost all operators | ||
| use std::task::Poll; | ||
|
|
||
| use arrow::{error::ArrowError, record_batch::RecordBatch}; | ||
|
|
||
| use super::{Count, ExecutionPlanMetricsSet, MetricBuilder, Time, Timestamp}; | ||
|
|
||
| /// Helper for creating and tracking common "baseline" metrics for | ||
|
Contributor
Author
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 is the core new structure / wrapper as suggested by @houqp -- it doesn't quite wrap everything yet but I think it makes annotating basic information pretty simple |
||
| /// each operator | ||
| /// | ||
| /// Example: | ||
| /// ``` | ||
| /// use datafusion::physical_plan::metrics::{BaselineMetrics, ExecutionPlanMetricsSet}; | ||
| /// let metrics = ExecutionPlanMetricsSet::new(); | ||
| /// | ||
| /// let partition = 2; | ||
| /// let baseline_metrics = BaselineMetrics::new(&metrics, partition); | ||
| /// | ||
| /// // during execution, in CPU intensive operation: | ||
| /// let timer = baseline_metrics.elapsed_compute().timer(); | ||
| /// // .. do CPU intensive work | ||
| /// timer.done(); | ||
| /// | ||
| /// // when operator is finished: | ||
| /// baseline_metrics.done(); | ||
| /// ``` | ||
| #[derive(Debug)] | ||
| pub struct BaselineMetrics { | ||
| /// end_time is set when `ExecutionMetrics::done()` is called | ||
| end_time: Timestamp, | ||
|
|
||
| /// amount of time the operator was actively trying to use the CPU | ||
| elapsed_compute: Time, | ||
|
|
||
| /// output rows: the total output rows | ||
| output_rows: Count, | ||
| } | ||
|
|
||
| impl BaselineMetrics { | ||
| /// Create a new BaselineMetric structure, and set `start_time` to now | ||
| pub fn new(metrics: &ExecutionPlanMetricsSet, partition: usize) -> Self { | ||
| let start_time = MetricBuilder::new(metrics).start_timestamp(partition); | ||
| start_time.record(); | ||
|
|
||
| Self { | ||
| end_time: MetricBuilder::new(metrics).end_timestamp(partition), | ||
| elapsed_compute: MetricBuilder::new(metrics).elapsed_compute(partition), | ||
| output_rows: MetricBuilder::new(metrics).output_rows(partition), | ||
| } | ||
| } | ||
|
|
||
| /// return the metric for cpu time spend in this operator | ||
| pub fn elapsed_compute(&self) -> &Time { | ||
| &self.elapsed_compute | ||
| } | ||
|
|
||
| /// return the metric for the total number of output rows produced | ||
| pub fn output_rows(&self) -> &Count { | ||
| &self.output_rows | ||
| } | ||
|
|
||
| /// Records the fact that this operator's execution is complete | ||
| /// (recording the `end_time` metric). | ||
| /// | ||
| /// Note care should be taken to call `done()` maually if | ||
| /// `BaselineMetrics` is not `drop`ped immediately upon operator | ||
| /// completion, as async streams may not be dropped immediately | ||
| /// depending on the consumer. | ||
| pub fn done(&self) { | ||
| self.end_time.record() | ||
| } | ||
|
|
||
| /// Record that some number of rows have been produced as output | ||
| /// | ||
| /// See the [`RecordOutput`] for conveniently recording record | ||
| /// batch output for other thing | ||
| pub fn record_output(&self, num_rows: usize) { | ||
| self.output_rows.add(num_rows); | ||
| } | ||
|
|
||
| /// Process a poll result of a stream producing output for an | ||
| /// operator, recording the output rows and stream done time and | ||
| /// returning the same poll result | ||
| pub fn record_poll( | ||
| &self, | ||
| poll: Poll<Option<Result<RecordBatch, ArrowError>>>, | ||
| ) -> Poll<Option<Result<RecordBatch, ArrowError>>> { | ||
| if let Poll::Ready(maybe_batch) = &poll { | ||
| match maybe_batch { | ||
| Some(Ok(batch)) => { | ||
| batch.record_output(self); | ||
| } | ||
| Some(Err(_)) => self.done(), | ||
| None => self.done(), | ||
| } | ||
| } | ||
| poll | ||
| } | ||
| } | ||
|
|
||
| impl Drop for BaselineMetrics { | ||
| fn drop(&mut self) { | ||
| // if not previously recorded, record | ||
| if self.end_time.value().is_none() { | ||
| self.end_time.record() | ||
| } | ||
| } | ||
| } | ||
|
|
||
| /// Trait for things that produce output rows as a result of execution. | ||
| pub trait RecordOutput { | ||
| /// Record that some number of output rows have been produced | ||
| /// | ||
| /// Meant to be composable so that instead of returning `batch` | ||
| /// the operator can return `batch.record_output(baseline_metrics)` | ||
| fn record_output(self, bm: &BaselineMetrics) -> Self; | ||
| } | ||
|
|
||
| impl RecordOutput for usize { | ||
| fn record_output(self, bm: &BaselineMetrics) -> Self { | ||
| bm.record_output(self); | ||
| self | ||
| } | ||
| } | ||
|
|
||
| impl RecordOutput for RecordBatch { | ||
| fn record_output(self, bm: &BaselineMetrics) -> Self { | ||
| bm.record_output(self.num_rows()); | ||
| self | ||
| } | ||
| } | ||
|
|
||
| impl RecordOutput for &RecordBatch { | ||
| fn record_output(self, bm: &BaselineMetrics) -> Self { | ||
| bm.record_output(self.num_rows()); | ||
| self | ||
| } | ||
| } | ||
|
|
||
| impl RecordOutput for Option<&RecordBatch> { | ||
| fn record_output(self, bm: &BaselineMetrics) -> Self { | ||
| if let Some(record_batch) = &self { | ||
| record_batch.record_output(bm); | ||
| } | ||
| self | ||
| } | ||
| } | ||
|
|
||
| impl RecordOutput for Option<RecordBatch> { | ||
| fn record_output(self, bm: &BaselineMetrics) -> Self { | ||
| if let Some(record_batch) = &self { | ||
| record_batch.record_output(bm); | ||
| } | ||
| self | ||
| } | ||
| } | ||
|
|
||
| impl RecordOutput for arrow::error::Result<RecordBatch> { | ||
| fn record_output(self, bm: &BaselineMetrics) -> Self { | ||
| if let Ok(record_batch) = &self { | ||
| record_batch.record_output(bm); | ||
| } | ||
| self | ||
| } | ||
| } | ||
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
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.
Uh oh!
There was an error while loading. Please reload this page.
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.
This is the pattern that instrumented operators now follow:
metrics: ExecutionPlanMetricsSetfieldexecute()they create newBaselineMetricselapsed_computetimer during their CPU intensive work