|
| 1 | +// Licensed to the Apache Software Foundation (ASF) under one |
| 2 | +// or more contributor license agreements. See the NOTICE file |
| 3 | +// distributed with this work for additional information |
| 4 | +// regarding copyright ownership. The ASF licenses this file |
| 5 | +// to you under the Apache License, Version 2.0 (the |
| 6 | +// "License"); you may not use this file except in compliance |
| 7 | +// with the License. You may obtain a copy of the License at |
| 8 | +// |
| 9 | +// http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | +// |
| 11 | +// Unless required by applicable law or agreed to in writing, |
| 12 | +// software distributed under the License is distributed on an |
| 13 | +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY |
| 14 | +// KIND, either express or implied. See the License for the |
| 15 | +// specific language governing permissions and limitations |
| 16 | +// under the License. |
| 17 | + |
| 18 | +//! Benchmark comparing incremental emission vs all-at-once emission |
| 19 | +//! for hash aggregation. |
| 20 | +//! |
| 21 | +//! This benchmark measures the time-to-first-row improvement from |
| 22 | +//! the incremental drain optimization. |
| 23 | +//! |
| 24 | +//! Usage: |
| 25 | +//! cargo run --release --bin incremental_emit_bench -- --groups 1000000 |
| 26 | +
|
| 27 | +use arrow::array::{Int64Array, StringArray}; |
| 28 | +use arrow::datatypes::{DataType, Field, Schema}; |
| 29 | +use arrow::record_batch::RecordBatch; |
| 30 | +use datafusion::common::Result; |
| 31 | +use datafusion::datasource::MemTable; |
| 32 | +use datafusion::prelude::*; |
| 33 | +use datafusion_common::instant::Instant; |
| 34 | +use futures::StreamExt; |
| 35 | +use std::sync::Arc; |
| 36 | +use std::time::Duration; |
| 37 | + |
| 38 | +#[derive(Debug, Clone)] |
| 39 | +struct BenchmarkResult { |
| 40 | + time_to_first_row: Duration, |
| 41 | + total_time: Duration, |
| 42 | + num_output_batches: usize, |
| 43 | + total_output_rows: usize, |
| 44 | +} |
| 45 | + |
| 46 | +async fn run_aggregation_benchmark( |
| 47 | + batch: RecordBatch, |
| 48 | + batch_size: usize, |
| 49 | +) -> Result<BenchmarkResult> { |
| 50 | + let config = SessionConfig::new().with_batch_size(batch_size); |
| 51 | + let ctx = SessionContext::new_with_config(config); |
| 52 | + |
| 53 | + let schema = batch.schema(); |
| 54 | + let table = MemTable::try_new(schema, vec![vec![batch]])?; |
| 55 | + ctx.register_table("bench_data", Arc::new(table))?; |
| 56 | + |
| 57 | + let sql = "SELECT group_key, SUM(value) as total, COUNT(*) as cnt \ |
| 58 | + FROM bench_data \ |
| 59 | + GROUP BY group_key"; |
| 60 | + |
| 61 | + let df = ctx.sql(sql).await?; |
| 62 | + |
| 63 | + let start = Instant::now(); |
| 64 | + let mut stream = df.execute_stream().await?; |
| 65 | + |
| 66 | + // Measure time to first batch |
| 67 | + let first_batch = stream.next().await; |
| 68 | + let time_to_first_row = start.elapsed(); |
| 69 | + |
| 70 | + let mut num_output_batches = 0; |
| 71 | + let mut total_output_rows = 0; |
| 72 | + |
| 73 | + if let Some(Ok(batch)) = first_batch { |
| 74 | + num_output_batches += 1; |
| 75 | + total_output_rows += batch.num_rows(); |
| 76 | + } |
| 77 | + |
| 78 | + // Consume remaining batches |
| 79 | + while let Some(result) = stream.next().await { |
| 80 | + if let Ok(batch) = result { |
| 81 | + num_output_batches += 1; |
| 82 | + total_output_rows += batch.num_rows(); |
| 83 | + } |
| 84 | + } |
| 85 | + |
| 86 | + let total_time = start.elapsed(); |
| 87 | + |
| 88 | + Ok(BenchmarkResult { |
| 89 | + time_to_first_row, |
| 90 | + total_time, |
| 91 | + num_output_batches, |
| 92 | + total_output_rows, |
| 93 | + }) |
| 94 | +} |
| 95 | + |
| 96 | +fn create_test_data(num_groups: usize, rows_per_group: usize) -> Result<RecordBatch> { |
| 97 | + let total_rows = num_groups * rows_per_group; |
| 98 | + |
| 99 | + // Create group keys: "group_0", "group_1", ..., "group_{num_groups-1}" |
| 100 | + // Each group appears rows_per_group times |
| 101 | + let group_keys: Vec<String> = (0..total_rows) |
| 102 | + .map(|i| format!("group_{}", i % num_groups)) |
| 103 | + .collect(); |
| 104 | + |
| 105 | + // Create values: just use the row index |
| 106 | + let values: Vec<i64> = (0..total_rows as i64).collect(); |
| 107 | + |
| 108 | + let schema = Arc::new(Schema::new(vec![ |
| 109 | + Field::new("group_key", DataType::Utf8, false), |
| 110 | + Field::new("value", DataType::Int64, false), |
| 111 | + ])); |
| 112 | + |
| 113 | + let batch = RecordBatch::try_new( |
| 114 | + schema, |
| 115 | + vec![ |
| 116 | + Arc::new(StringArray::from(group_keys)), |
| 117 | + Arc::new(Int64Array::from(values)), |
| 118 | + ], |
| 119 | + )?; |
| 120 | + |
| 121 | + Ok(batch) |
| 122 | +} |
| 123 | + |
| 124 | +#[tokio::main] |
| 125 | +async fn main() -> Result<()> { |
| 126 | + let args: Vec<String> = std::env::args().collect(); |
| 127 | + |
| 128 | + let num_groups = args |
| 129 | + .iter() |
| 130 | + .position(|s| s == "--groups") |
| 131 | + .and_then(|i| args.get(i + 1)) |
| 132 | + .and_then(|s| s.parse().ok()) |
| 133 | + .unwrap_or(100_000); |
| 134 | + |
| 135 | + let rows_per_group = args |
| 136 | + .iter() |
| 137 | + .position(|s| s == "--rows-per-group") |
| 138 | + .and_then(|i| args.get(i + 1)) |
| 139 | + .and_then(|s| s.parse().ok()) |
| 140 | + .unwrap_or(10); |
| 141 | + |
| 142 | + let iterations = args |
| 143 | + .iter() |
| 144 | + .position(|s| s == "--iterations") |
| 145 | + .and_then(|i| args.get(i + 1)) |
| 146 | + .and_then(|s| s.parse().ok()) |
| 147 | + .unwrap_or(3); |
| 148 | + |
| 149 | + println!("=== Incremental Emit Benchmark ==="); |
| 150 | + println!("Number of groups: {num_groups}"); |
| 151 | + println!("Rows per group: {rows_per_group}"); |
| 152 | + println!("Total input rows: {}", num_groups * rows_per_group); |
| 153 | + println!("Iterations: {iterations}"); |
| 154 | + println!(); |
| 155 | + |
| 156 | + // Batch sizes to test |
| 157 | + // Note: We use num_groups as the "all-at-once" batch size to simulate the EmitTo::All behavior |
| 158 | + // of emitting all groups in a single batch. |
| 159 | + let batch_sizes = vec![ |
| 160 | + (8192, "8192 (incremental)".to_string()), |
| 161 | + (32768, "32768 (larger batches)".to_string()), |
| 162 | + ( |
| 163 | + num_groups, |
| 164 | + format!("{num_groups} (all-at-once, simulates EmitTo::All behavior)"), |
| 165 | + ), |
| 166 | + ]; |
| 167 | + |
| 168 | + println!("Running benchmarks..."); |
| 169 | + println!(); |
| 170 | + |
| 171 | + for (batch_size, label) in batch_sizes { |
| 172 | + println!("--- Batch size: {label} ---"); |
| 173 | + |
| 174 | + let mut first_row_times = Vec::new(); |
| 175 | + let mut total_times = Vec::new(); |
| 176 | + |
| 177 | + for i in 0..iterations { |
| 178 | + // Create fresh test data for each run |
| 179 | + let batch = create_test_data(num_groups, rows_per_group)?; |
| 180 | + let result = run_aggregation_benchmark(batch, batch_size).await?; |
| 181 | + |
| 182 | + println!( |
| 183 | + " Iteration {}: first_row={:?}, total={:?}, batches={}, rows={}", |
| 184 | + i + 1, |
| 185 | + result.time_to_first_row, |
| 186 | + result.total_time, |
| 187 | + result.num_output_batches, |
| 188 | + result.total_output_rows |
| 189 | + ); |
| 190 | + |
| 191 | + first_row_times.push(result.time_to_first_row); |
| 192 | + total_times.push(result.total_time); |
| 193 | + } |
| 194 | + |
| 195 | + let avg_first_row: Duration = |
| 196 | + first_row_times.iter().sum::<Duration>() / iterations as u32; |
| 197 | + let avg_total: Duration = |
| 198 | + total_times.iter().sum::<Duration>() / iterations as u32; |
| 199 | + |
| 200 | + println!(" Average: first_row={avg_first_row:?}, total={avg_total:?}"); |
| 201 | + println!(); |
| 202 | + } |
| 203 | + |
| 204 | + println!("=== Summary ==="); |
| 205 | + println!("The 'time to first row' metric shows how quickly the first output"); |
| 206 | + println!("batch is produced. With incremental emission (smaller batch sizes),"); |
| 207 | + println!("this should be significantly faster than all-at-once emission."); |
| 208 | + |
| 209 | + Ok(()) |
| 210 | +} |
| 211 | + |
| 212 | +/* Example output: |
| 213 | +
|
| 214 | +cargo run --bin incremental_emit -- --groups 1000000 --rows-per-group 5 --iterations 3 |
| 215 | +
|
| 216 | +=== Incremental Emit Benchmark === |
| 217 | +Number of groups: 1000000 |
| 218 | +Rows per group: 5 |
| 219 | +Total input rows: 5000000 |
| 220 | +Iterations: 3 |
| 221 | +
|
| 222 | +Running benchmarks... |
| 223 | +
|
| 224 | +--- Batch size: 8192 (incremental) --- |
| 225 | + Iteration 1: first_row=514.312458ms, total=750.121625ms, batches=128, rows=1000000 |
| 226 | + Iteration 2: first_row=487.098583ms, total=680.311958ms, batches=128, rows=1000000 |
| 227 | + Iteration 3: first_row=473.02925ms, total=668.469083ms, batches=128, rows=1000000 |
| 228 | + Average: first_row=491.480097ms, total=699.634222ms |
| 229 | +
|
| 230 | +--- Batch size: 32768 (larger batches) --- |
| 231 | + Iteration 1: first_row=481.137417ms, total=497.485917ms, batches=32, rows=1000000 |
| 232 | + Iteration 2: first_row=478.821ms, total=496.062959ms, batches=32, rows=1000000 |
| 233 | + Iteration 3: first_row=524.281709ms, total=539.426584ms, batches=32, rows=1000000 |
| 234 | + Average: first_row=494.746708ms, total=510.99182ms |
| 235 | +
|
| 236 | +--- Batch size: 1000000 (all-at-once, simulates EmitTo::All behavior) --- |
| 237 | + Iteration 1: first_row=1.22554625s, total=1.303929583s, batches=16, rows=1000000 |
| 238 | + Iteration 2: first_row=1.237296333s, total=1.2897605s, batches=16, rows=1000000 |
| 239 | + Iteration 3: first_row=1.235812417s, total=1.303563667s, batches=16, rows=1000000 |
| 240 | + Average: first_row=1.232885s, total=1.299084583s |
| 241 | +
|
| 242 | +=== Summary === |
| 243 | +The 'time to first row' metric shows how quickly the first output |
| 244 | +batch is produced. With incremental emission (smaller batch sizes), |
| 245 | +this should be significantly faster than all-at-once emission. |
| 246 | +*/ |
0 commit comments