Add a new metric type: Gauge + CurrentMemoryUsage to metrics#1682
Add a new metric type: Gauge + CurrentMemoryUsage to metrics#1682alamb merged 4 commits intoapache:masterfrom
Gauge + CurrentMemoryUsage to metrics#1682Conversation
| impl AggregatedMetricsSet { | ||
| /// Create a new aggregated set | ||
| pub(crate) fn new() -> Self { | ||
| pub fn new() -> Self { |
There was a problem hiding this comment.
To create a memory-managed version of ShuffleWriter, we need to expose these four APIs for dependent crate usage.
I'm not sure this should be in its own PR, or I can have it here since this PR is also metric-related. I can remove this if needed.
There was a problem hiding this comment.
I think it is fine to be in this PR
datafusion/src/physical_plan/mod.rs
Outdated
| /// Returns the current memory usage for this stream. | ||
| fn mem_used(&self) -> usize { | ||
| 0 | ||
| } |
There was a problem hiding this comment.
This line adds a mem_used method in our essential RecordBatchStream trait.
A baby step to tracking Non-Limited-Operators' memory usage since I think SendableRecordBatchStream is the fundamental entity that holds memory during execution. However, I didn't quite find a way to register these streams generated during async execute to our memory manager.
I would love to hear your thoughts.
If considered not appropriate, I will remove it.
There was a problem hiding this comment.
After re-consider this for a while. I cannot think of a solution to register RecordBatchStream somewhere else since each stream is used through mutable reference. I don't think there's a way sharing it except for Arc<dyn RecordBatchStream ....>, which we have discussed earlier and is not acceptable.
I'm going to revert this last commit.
GaugeGauge + CurrentMemoryUsage to metrics
| impl AggregatedMetricsSet { | ||
| /// Create a new aggregated set | ||
| pub(crate) fn new() -> Self { | ||
| pub fn new() -> Self { |
There was a problem hiding this comment.
I think it is fine to be in this PR
| use std::sync::Arc; | ||
| use tempfile::NamedTempFile; | ||
| use tokio::sync::mpsc::{Receiver as TKReceiver, Sender as TKSender}; | ||
| use tokio::sync::mpsc::{Receiver, Sender}; |
|
And thank you for the review @liukun4515 |
Which issue does this PR close?
Closes #.
Rationale for this change
A gauge metric is useful for reporting purposes by returning a value each time, and a gauge suits well for memory consumption reports.
What changes are included in this PR?
GaugeCurrentMemoryUsagetoBaselineMetricsgaugein sort for memory tracking.Are there any user-facing changes?
No.