-
Notifications
You must be signed in to change notification settings - Fork 2k
Add metrics for parquet sink #20307
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
Add metrics for parquet sink #20307
Changes from all commits
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 |
|---|---|---|
|
|
@@ -55,6 +55,9 @@ use datafusion_execution::memory_pool::{MemoryConsumer, MemoryPool, MemoryReserv | |
| use datafusion_execution::{SendableRecordBatchStream, TaskContext}; | ||
| use datafusion_expr::dml::InsertOp; | ||
| use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; | ||
| use datafusion_physical_plan::metrics::{ | ||
| ExecutionPlanMetricsSet, MetricBuilder, MetricsSet, | ||
| }; | ||
| use datafusion_physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan}; | ||
| use datafusion_session::Session; | ||
|
|
||
|
|
@@ -1156,6 +1159,8 @@ pub struct ParquetSink { | |
| written: Arc<parking_lot::Mutex<HashMap<Path, ParquetMetaData>>>, | ||
| /// Optional sorting columns to write to Parquet metadata | ||
| sorting_columns: Option<Vec<SortingColumn>>, | ||
| /// Metrics for tracking write operations | ||
| metrics: ExecutionPlanMetricsSet, | ||
| } | ||
|
|
||
| impl Debug for ParquetSink { | ||
|
|
@@ -1188,6 +1193,7 @@ impl ParquetSink { | |
| parquet_options, | ||
| written: Default::default(), | ||
| sorting_columns: None, | ||
| metrics: ExecutionPlanMetricsSet::new(), | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -1333,6 +1339,17 @@ impl FileSink for ParquetSink { | |
| mut file_stream_rx: DemuxedStreamReceiver, | ||
| object_store: Arc<dyn ObjectStore>, | ||
| ) -> Result<u64> { | ||
| let rows_written_counter = | ||
| MetricBuilder::new(&self.metrics).global_counter("rows_written"); | ||
| // Note: bytes_written is the sum of compressed row group sizes, which | ||
| // may differ slightly from the actual on-disk file size (excludes footer, | ||
| // page indexes, and other Parquet metadata overhead). | ||
| let bytes_written_counter = | ||
| MetricBuilder::new(&self.metrics).global_counter("bytes_written"); | ||
| let elapsed_compute = MetricBuilder::new(&self.metrics).elapsed_compute(0); | ||
|
|
||
| let write_start = datafusion_common::instant::Instant::now(); | ||
|
|
||
| let parquet_opts = &self.parquet_options; | ||
|
|
||
| let mut file_write_tasks: JoinSet< | ||
|
|
@@ -1410,12 +1427,18 @@ impl FileSink for ParquetSink { | |
| } | ||
| } | ||
|
|
||
| let mut row_count = 0; | ||
| while let Some(result) = file_write_tasks.join_next().await { | ||
| match result { | ||
| Ok(r) => { | ||
| let (path, parquet_meta_data) = r?; | ||
| row_count += parquet_meta_data.file_metadata().num_rows(); | ||
| let file_rows = parquet_meta_data.file_metadata().num_rows() as usize; | ||
| let file_bytes: usize = parquet_meta_data | ||
| .row_groups() | ||
| .iter() | ||
| .map(|rg| rg.compressed_size() as usize) | ||
| .sum(); | ||
| rows_written_counter.add(file_rows); | ||
| bytes_written_counter.add(file_bytes); | ||
|
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. Given that bytes_written is derived from row_groups().compressed_size() and may not reflect the exact on-disk file size, should we clarify this in the documentation or rename it to something like compressed_row_group_bytes?
Member
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. Good point, I think (Renaming to |
||
| let mut written_files = self.written.lock(); | ||
| written_files | ||
| .try_insert(path.clone(), parquet_meta_data) | ||
|
|
@@ -1437,7 +1460,9 @@ impl FileSink for ParquetSink { | |
| .await | ||
| .map_err(|e| DataFusionError::ExecutionJoin(Box::new(e)))??; | ||
|
|
||
| Ok(row_count as u64) | ||
| elapsed_compute.add_elapsed(write_start); | ||
|
|
||
| Ok(rows_written_counter.value() as u64) | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -1447,6 +1472,10 @@ impl DataSink for ParquetSink { | |
| self | ||
| } | ||
|
|
||
| fn metrics(&self) -> Option<MetricsSet> { | ||
| Some(self.metrics.clone_inner()) | ||
| } | ||
|
|
||
| fn schema(&self) -> &SchemaRef { | ||
| self.config.output_schema() | ||
| } | ||
|
|
||
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.
Since Parquet now implements sink metrics but CSV/JSON still use the default DataSink::metrics() -> None, is this divergence intentional? If so, would it make sense to follow up by centralizing common sink metrics wiring (e.g., in FileSink or write orchestration) to reduce duplication and enable consistent opt-in across file sinks?
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.
Makes sense, I'll open an issue and make a follow-up PR for this.
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.
opened an issue #20644