Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
214 changes: 214 additions & 0 deletions datafusion/physical-plan/src/aggregates/group_values/metrics.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,214 @@
// 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 for the various group-by implementations.

use crate::metrics::{ExecutionPlanMetricsSet, MetricBuilder, Time};

pub(crate) struct GroupByMetrics {
/// Time spent calculating the group IDs from the evaluated grouping columns.
pub(crate) time_calculating_group_ids: Time,
/// Time spent evaluating the inputs to the aggregate functions.
pub(crate) aggregate_arguments_time: Time,
/// Time spent evaluating the aggregate expressions themselves
/// (e.g. summing all elements and counting number of elements for `avg` aggregate).
pub(crate) aggregation_time: Time,
/// Time spent emitting the final results and constructing the record batch
/// which includes finalizing the grouping expressions
/// (e.g. emit from the hash table in case of hash aggregation) and the accumulators
pub(crate) emitting_time: Time,
}

impl GroupByMetrics {
pub(crate) fn new(metrics: &ExecutionPlanMetricsSet, partition: usize) -> Self {
Self {
time_calculating_group_ids: MetricBuilder::new(metrics)
.subset_time("time_calculating_group_ids", partition),
aggregate_arguments_time: MetricBuilder::new(metrics)
.subset_time("aggregate_arguments_time", partition),
aggregation_time: MetricBuilder::new(metrics)
.subset_time("aggregation_time", partition),
emitting_time: MetricBuilder::new(metrics)
.subset_time("emitting_time", partition),
}
}
}

#[cfg(test)]
mod tests {
use crate::aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy};
use crate::metrics::MetricsSet;
use crate::test::TestMemoryExec;
use crate::{collect, ExecutionPlan};
use arrow::array::{Float64Array, UInt32Array};
use arrow::datatypes::{DataType, Field, Schema};
use arrow::record_batch::RecordBatch;
use datafusion_common::Result;
use datafusion_execution::TaskContext;
use datafusion_functions_aggregate::count::count_udaf;
use datafusion_functions_aggregate::sum::sum_udaf;
use datafusion_physical_expr::aggregate::AggregateExprBuilder;
use datafusion_physical_expr::expressions::col;
use std::sync::Arc;

/// Helper function to verify all three GroupBy metrics exist and have non-zero values
fn assert_groupby_metrics(metrics: &MetricsSet) {
let agg_arguments_time = metrics.sum_by_name("aggregate_arguments_time");
assert!(agg_arguments_time.is_some());
assert!(agg_arguments_time.unwrap().as_usize() > 0);

let aggregation_time = metrics.sum_by_name("aggregation_time");
assert!(aggregation_time.is_some());
assert!(aggregation_time.unwrap().as_usize() > 0);

let emitting_time = metrics.sum_by_name("emitting_time");
assert!(emitting_time.is_some());
assert!(emitting_time.unwrap().as_usize() > 0);
}

#[tokio::test]
async fn test_groupby_metrics_partial_mode() -> Result<()> {
let schema = Arc::new(Schema::new(vec![
Field::new("a", DataType::UInt32, false),
Field::new("b", DataType::Float64, false),
]));

// Create multiple batches to ensure metrics accumulate
let batches = (0..5)
.map(|i| {
RecordBatch::try_new(
Arc::clone(&schema),
vec![
Arc::new(UInt32Array::from(vec![1, 2, 3, 4])),
Arc::new(Float64Array::from(vec![
i as f64,
(i + 1) as f64,
(i + 2) as f64,
(i + 3) as f64,
])),
],
)
.unwrap()
})
.collect::<Vec<_>>();

let input = TestMemoryExec::try_new_exec(&[batches], Arc::clone(&schema), None)?;

let group_by =
PhysicalGroupBy::new_single(vec![(col("a", &schema)?, "a".to_string())]);

let aggregates = vec![
Arc::new(
AggregateExprBuilder::new(sum_udaf(), vec![col("b", &schema)?])
.schema(Arc::clone(&schema))
.alias("SUM(b)")
.build()?,
),
Arc::new(
AggregateExprBuilder::new(count_udaf(), vec![col("b", &schema)?])
.schema(Arc::clone(&schema))
.alias("COUNT(b)")
.build()?,
),
];

let aggregate_exec = Arc::new(AggregateExec::try_new(
AggregateMode::Partial,
group_by,
aggregates,
vec![None, None],
input,
schema,
)?);

let task_ctx = Arc::new(TaskContext::default());
let _result =
collect(Arc::clone(&aggregate_exec) as _, Arc::clone(&task_ctx)).await?;

let metrics = aggregate_exec.metrics().unwrap();
assert_groupby_metrics(&metrics);

Ok(())
}

#[tokio::test]
async fn test_groupby_metrics_final_mode() -> Result<()> {
let schema = Arc::new(Schema::new(vec![
Field::new("a", DataType::UInt32, false),
Field::new("b", DataType::Float64, false),
]));

let batches = (0..3)
.map(|i| {
RecordBatch::try_new(
Arc::clone(&schema),
vec![
Arc::new(UInt32Array::from(vec![1, 2, 3])),
Arc::new(Float64Array::from(vec![
i as f64,
(i + 1) as f64,
(i + 2) as f64,
])),
],
)
.unwrap()
})
.collect::<Vec<_>>();

let partial_input =
TestMemoryExec::try_new_exec(&[batches], Arc::clone(&schema), None)?;

let group_by =
PhysicalGroupBy::new_single(vec![(col("a", &schema)?, "a".to_string())]);

let aggregates = vec![Arc::new(
AggregateExprBuilder::new(sum_udaf(), vec![col("b", &schema)?])
.schema(Arc::clone(&schema))
.alias("SUM(b)")
.build()?,
)];

// Create partial aggregate
let partial_aggregate = Arc::new(AggregateExec::try_new(
AggregateMode::Partial,
group_by.clone(),
aggregates.clone(),
vec![None],
partial_input,
Arc::clone(&schema),
)?);

// Create final aggregate
let final_aggregate = Arc::new(AggregateExec::try_new(
AggregateMode::Final,
group_by.as_final(),
aggregates,
vec![None],
partial_aggregate,
schema,
)?);

let task_ctx = Arc::new(TaskContext::default());
let _result =
collect(Arc::clone(&final_aggregate) as _, Arc::clone(&task_ctx)).await?;

let metrics = final_aggregate.metrics().unwrap();
assert_groupby_metrics(&metrics);

Ok(())
}
}
3 changes: 3 additions & 0 deletions datafusion/physical-plan/src/aggregates/group_values/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -46,8 +46,11 @@ use crate::aggregates::{
order::GroupOrdering,
};

mod metrics;
mod null_builder;

pub(crate) use metrics::GroupByMetrics;

/// Stores the group values during hash aggregation.
///
/// # Background
Expand Down
35 changes: 34 additions & 1 deletion datafusion/physical-plan/src/aggregates/row_hash.rs
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ use std::vec;

use super::order::GroupOrdering;
use super::AggregateExec;
use crate::aggregates::group_values::{new_group_values, GroupValues};
use crate::aggregates::group_values::{new_group_values, GroupByMetrics, GroupValues};
use crate::aggregates::order::GroupOrderingFull;
use crate::aggregates::{
create_schema, evaluate_group_by, evaluate_many, evaluate_optional, AggregateMode,
Expand All @@ -49,6 +49,7 @@ use datafusion_physical_expr::expressions::Column;
use datafusion_physical_expr::{GroupsAccumulatorAdapter, PhysicalSortExpr};
use datafusion_physical_expr_common::sort_expr::LexOrdering;

use datafusion_common::instant::Instant;
use futures::ready;
use futures::stream::{Stream, StreamExt};
use log::debug;
Expand Down Expand Up @@ -430,6 +431,9 @@ pub(crate) struct GroupedHashAggregateStream {

/// Execution metrics
baseline_metrics: BaselineMetrics,

/// Aggregation-specific metrics
group_by_metrics: GroupByMetrics,
Copy link
Member

@rluvaton rluvaton Oct 28, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can you also add a timer for building and searching (single timer) the the hash table (group_values.intern)

or if you prefer we can leave that for later

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this will become a performance hit, I'd rather keep metrics concrete to avoid syscalls

}

impl GroupedHashAggregateStream {
Expand All @@ -447,6 +451,7 @@ impl GroupedHashAggregateStream {
let batch_size = context.session_config().batch_size();
let input = agg.input.execute(partition, Arc::clone(&context))?;
let baseline_metrics = BaselineMetrics::new(&agg.metrics, partition);
let group_by_metrics = GroupByMetrics::new(&agg.metrics, partition);

let timer = baseline_metrics.elapsed_compute().timer();

Expand Down Expand Up @@ -609,6 +614,7 @@ impl GroupedHashAggregateStream {
current_group_indices: Default::default(),
exec_state,
baseline_metrics,
group_by_metrics,
batch_size,
group_ordering,
input_done: false,
Expand Down Expand Up @@ -830,12 +836,25 @@ impl GroupedHashAggregateStream {
evaluate_group_by(&self.group_by, &batch)?
};

// Only create the timer if there are actual aggregate arguments to evaluate
let timer = match (
self.spill_state.is_stream_merging,
self.spill_state.merging_aggregate_arguments.is_empty(),
self.aggregate_arguments.is_empty(),
) {
(true, false, _) | (false, _, false) => {
Some(self.group_by_metrics.aggregate_arguments_time.timer())
}
_ => None,
};

// Evaluate the aggregation expressions.
let input_values = if self.spill_state.is_stream_merging {
evaluate_many(&self.spill_state.merging_aggregate_arguments, &batch)?
} else {
evaluate_many(&self.aggregate_arguments, &batch)?
};
drop(timer);

// Evaluate the filter expressions, if any, against the inputs
let filter_values = if self.spill_state.is_stream_merging {
Expand All @@ -846,6 +865,8 @@ impl GroupedHashAggregateStream {
};

for group_values in &group_by_values {
let groups_start_time = Instant::now();

// calculate the group indices for each input row
let starting_num_groups = self.group_values.len();
self.group_values
Expand All @@ -862,6 +883,12 @@ impl GroupedHashAggregateStream {
)?;
}

// Use this instant for both measurements to save a syscall
let agg_start_time = Instant::now();
self.group_by_metrics
.time_calculating_group_ids
.add_duration(agg_start_time - groups_start_time);

// Gather the inputs to call the actual accumulator
let t = self
.accumulators
Expand Down Expand Up @@ -897,6 +924,9 @@ impl GroupedHashAggregateStream {
acc.merge_batch(values, group_indices, None, total_num_groups)?;
}
}
self.group_by_metrics
.aggregation_time
.add_elapsed(agg_start_time);
}
}

Expand Down Expand Up @@ -941,6 +971,7 @@ impl GroupedHashAggregateStream {
return Ok(None);
}

let timer = self.group_by_metrics.emitting_time.timer();
let mut output = self.group_values.emit(emit_to)?;
if let EmitTo::First(n) = emit_to {
self.group_ordering.remove_groups(n);
Expand All @@ -961,12 +992,14 @@ impl GroupedHashAggregateStream {
| AggregateMode::SinglePartitioned => output.push(acc.evaluate(emit_to)?),
}
}
drop(timer);

// emit reduces the memory usage. Ignore Err from update_memory_reservation. Even if it is
// over the target memory size after emission, we can emit again rather than returning Err.
let _ = self.update_memory_reservation();
let batch = RecordBatch::try_new(schema, output)?;
debug_assert!(batch.num_rows() > 0);

Ok(Some(batch))
}

Expand Down
Loading