forked from apache/datafusion
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathaggregate_fuzz.rs
581 lines (526 loc) · 19.1 KB
/
aggregate_fuzz.rs
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
// 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.
use std::sync::Arc;
use arrow::array::{Array, ArrayRef, AsArray, Int64Array};
use arrow::compute::{concat_batches, SortOptions};
use arrow::datatypes::DataType;
use arrow::record_batch::RecordBatch;
use arrow::util::pretty::pretty_format_batches;
use arrow_array::types::Int64Type;
use datafusion::common::Result;
use datafusion::datasource::MemTable;
use datafusion::physical_expr::aggregate::AggregateExprBuilder;
use datafusion::physical_plan::aggregates::{
AggregateExec, AggregateMode, PhysicalGroupBy,
};
use datafusion::physical_plan::memory::MemoryExec;
use datafusion::physical_plan::{collect, displayable, ExecutionPlan};
use datafusion::prelude::{DataFrame, SessionConfig, SessionContext};
use datafusion_common::tree_node::{TreeNode, TreeNodeRecursion, TreeNodeVisitor};
use datafusion_common::ScalarValue;
use datafusion_execution::disk_manager::DiskManagerConfig;
use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv};
use datafusion_execution::TaskContext;
use datafusion_functions_aggregate::sum::sum_udaf;
use datafusion_physical_expr::expressions::col;
use datafusion_physical_expr::PhysicalSortExpr;
use datafusion_physical_plan::InputOrderMode;
use rand::seq::SliceRandom;
use test_utils::{add_empty_batches, StringBatchGenerator};
use hashbrown::HashMap;
use rand::rngs::StdRng;
use rand::{thread_rng, Rng, SeedableRng};
use tokio::task::JoinSet;
/// Tests that streaming aggregate and batch (non streaming) aggregate produce
/// same results
#[tokio::test(flavor = "multi_thread")]
async fn streaming_aggregate_test() {
let test_cases = [
vec!["a"],
vec!["b", "a"],
vec!["c", "a"],
vec!["c", "b", "a"],
vec!["d", "a"],
vec!["d", "b", "a"],
vec!["d", "c", "a"],
vec!["d", "c", "b", "a"],
];
let n = 300;
let distincts = vec![10, 20];
for distinct in distincts {
let mut join_set = JoinSet::new();
for i in 0..n {
let test_idx = i % test_cases.len();
let group_by_columns = test_cases[test_idx].clone();
join_set.spawn(run_streaming_aggregate_test(
make_staggered_batches::<true>(1000, distinct, i as u64),
group_by_columns,
));
}
while let Some(join_handle) = join_set.join_next().await {
// propagate errors
join_handle.unwrap();
}
}
}
/// Tests that streaming aggregate and batch (non streaming) aggregate produce
/// same results
#[tokio::test(flavor = "multi_thread")]
async fn blocked_approach_aggregate_test() {
let test_cases = [
vec!["a"],
vec!["b", "a"],
vec!["c", "a"],
vec!["c", "b", "a"],
vec!["d", "a"],
vec!["d", "b", "a"],
vec!["d", "c", "a"],
vec!["d", "c", "b", "a"],
];
let n_batch_size = 10;
let mut rng = thread_rng();
let mut all_batch_sizes = (1..=50_usize).collect::<Vec<_>>();
all_batch_sizes.shuffle(&mut rng);
let batch_sizes = &all_batch_sizes[0..n_batch_size];
let n = 300;
let distincts = vec![10, 20];
for distinct in distincts {
let mut join_set = JoinSet::new();
for batch_size in batch_sizes {
for i in 0..n {
let test_idx = i % test_cases.len();
let group_by_columns = test_cases[test_idx].clone();
join_set.spawn(run_blocked_approach_aggregate_test(
make_staggered_batches::<true>(1000, distinct, i as u64),
group_by_columns,
*batch_size,
));
}
}
while let Some(join_handle) = join_set.join_next().await {
// propagate errors
join_handle.unwrap();
}
}
}
/// Perform batch and streaming aggregation with same input
/// and verify outputs of `AggregateExec` with pipeline breaking stream `GroupedHashAggregateStream`
/// and non-pipeline breaking stream `BoundedAggregateStream` produces same result.
async fn run_streaming_aggregate_test(
test_data: Vec<RecordBatch>,
group_by_columns: Vec<&str>,
) {
let schema = test_data[0].schema();
// Define test data source exec
let mut sort_keys = vec![];
for ordering_col in ["a", "b", "c"] {
sort_keys.push(PhysicalSortExpr {
expr: col(ordering_col, &schema).unwrap(),
options: SortOptions::default(),
})
}
let concat_input_record = concat_batches(&schema, &test_data).unwrap();
let usual_source = Arc::new(
MemoryExec::try_new(&[vec![concat_input_record]], schema.clone(), None).unwrap(),
);
let running_source = Arc::new(
MemoryExec::try_new(&[test_data.clone()], schema.clone(), None)
.unwrap()
.with_sort_information(vec![sort_keys]),
);
// Define test task ctx
let session_config = SessionConfig::new().with_batch_size(50);
let ctx = SessionContext::new_with_config(session_config);
// Run and check
let usual_aggr_ctx = AggrTestContext {
data_source_exec: usual_source,
task_ctx: ctx.task_ctx(),
};
let running_aggr_ctx = AggrTestContext {
data_source_exec: running_source,
task_ctx: ctx.task_ctx(),
};
run_aggregate_test_internal(
test_data,
usual_aggr_ctx,
running_aggr_ctx,
|collected_usual, collected_running| {
assert!(collected_running.len() > 2);
// Running should produce more chunk than the usual AggregateExec.
// Otherwise it means that we cannot generate result in running mode.
assert!(collected_running.len() > collected_usual.len());
},
group_by_columns,
)
.await;
}
/// Perform batch and blocked approach aggregations, and then verify their outputs.
async fn run_blocked_approach_aggregate_test(
test_data: Vec<RecordBatch>,
group_by_columns: Vec<&str>,
batch_size: usize,
) {
let schema = test_data[0].schema();
// Define test data source exec
let concat_input_record = concat_batches(&schema, &test_data).unwrap();
let usual_source = Arc::new(
MemoryExec::try_new(&[vec![concat_input_record]], schema.clone(), None).unwrap(),
);
let running_source = Arc::new(
MemoryExec::try_new(&[test_data.clone()], schema.clone(), None).unwrap(),
);
// Define test task ctx
// Usual task ctx
let mut session_config = SessionConfig::default();
session_config = session_config.set(
"datafusion.execution.batch_size",
ScalarValue::UInt64(Some(batch_size as u64)),
);
let usual_ctx = Arc::new(TaskContext::default().with_session_config(session_config));
// Running task ctx
let mut session_config = SessionConfig::default();
session_config = session_config.set(
"datafusion.execution.enable_aggregation_group_states_blocked_approach",
ScalarValue::Boolean(Some(true)),
);
session_config = session_config.set(
"datafusion.execution.batch_size",
ScalarValue::UInt64(Some(batch_size as u64)),
);
let runtime = Arc::new(
RuntimeEnv::new(
RuntimeConfig::default().with_disk_manager(DiskManagerConfig::Disabled),
)
.unwrap(),
);
let running_ctx = Arc::new(
TaskContext::default()
.with_session_config(session_config)
.with_runtime(runtime),
);
// Run and check
let usual_aggr_ctx = AggrTestContext {
data_source_exec: usual_source,
task_ctx: usual_ctx,
};
let running_aggr_ctx = AggrTestContext {
data_source_exec: running_source,
task_ctx: running_ctx,
};
run_aggregate_test_internal(
test_data,
usual_aggr_ctx,
running_aggr_ctx,
|_, _| {},
group_by_columns,
)
.await;
}
/// Options of the fuzz aggregation tests
struct AggrTestContext {
data_source_exec: Arc<dyn ExecutionPlan>,
task_ctx: Arc<TaskContext>,
}
/// The internal test function for performing normal aggregation
/// and other optimized aggregations (without any optimizations,
/// e.g. streaming, blocked approach), and verify outputs of them.
async fn run_aggregate_test_internal<C>(
test_data: Vec<RecordBatch>,
left_aggr_ctx: AggrTestContext,
right_aggr_ctx: AggrTestContext,
extra_checks: C,
group_by_columns: Vec<&str>,
) where
C: Fn(&[RecordBatch], &[RecordBatch]),
{
let schema = test_data[0].schema();
let aggregate_expr =
vec![
AggregateExprBuilder::new(sum_udaf(), vec![col("d", &schema).unwrap()])
.schema(Arc::clone(&schema))
.alias("sum1")
.build()
.unwrap(),
];
let expr = group_by_columns
.iter()
.map(|elem| (col(elem, &schema).unwrap(), elem.to_string()))
.collect::<Vec<_>>();
let group_by = PhysicalGroupBy::new_single(expr);
let aggregate_exec_usual = Arc::new(
AggregateExec::try_new(
AggregateMode::Partial,
group_by.clone(),
aggregate_expr.clone(),
vec![None],
left_aggr_ctx.data_source_exec.clone(),
schema.clone(),
)
.unwrap(),
) as Arc<dyn ExecutionPlan>;
let aggregate_exec_running = Arc::new(
AggregateExec::try_new(
AggregateMode::Partial,
group_by.clone(),
aggregate_expr.clone(),
vec![None],
right_aggr_ctx.data_source_exec.clone(),
schema.clone(),
)
.unwrap(),
) as Arc<dyn ExecutionPlan>;
let collected_usual =
collect(aggregate_exec_usual.clone(), left_aggr_ctx.task_ctx.clone())
.await
.unwrap();
let collected_running = collect(
aggregate_exec_running.clone(),
right_aggr_ctx.task_ctx.clone(),
)
.await
.unwrap();
extra_checks(&collected_usual, &collected_running);
// compare
let usual_formatted = pretty_format_batches(&collected_usual).unwrap().to_string();
let running_formatted = pretty_format_batches(&collected_running)
.unwrap()
.to_string();
let mut usual_formatted_sorted: Vec<&str> = usual_formatted.trim().lines().collect();
usual_formatted_sorted.sort_unstable();
let mut running_formatted_sorted: Vec<&str> =
running_formatted.trim().lines().collect();
running_formatted_sorted.sort_unstable();
for (i, (usual_line, running_line)) in usual_formatted_sorted
.iter()
.zip(&running_formatted_sorted)
.enumerate()
{
assert_eq!(
(i, usual_line),
(i, running_line),
"Inconsistent result\n\n\
Aggregate_expr: {aggregate_expr:?}\n\
group_by: {group_by:?}\n\
Left Plan:\n{}\n\
Right Plan:\n{}\n\
schema:\n{schema}\n\
Left Ouptut:\n{}\n\
Right Output:\n{}\n\
input:\n{}\n\
",
displayable(aggregate_exec_usual.as_ref()).indent(false),
displayable(aggregate_exec_running.as_ref()).indent(false),
usual_formatted,
running_formatted,
pretty_format_batches(&test_data).unwrap(),
);
}
}
/// Return randomly sized record batches with:
/// three sorted int64 columns 'a', 'b', 'c' ranged from 0..'n_distinct' as columns
/// one random int64 column 'd' as other columns
pub(crate) fn make_staggered_batches<const STREAM: bool>(
len: usize,
n_distinct: usize,
random_seed: u64,
) -> Vec<RecordBatch> {
// use a random number generator to pick a random sized output
let mut rng = StdRng::seed_from_u64(random_seed);
let mut input123: Vec<(i64, i64, i64)> = vec![(0, 0, 0); len];
let mut input4: Vec<i64> = vec![0; len];
input123.iter_mut().for_each(|v| {
*v = (
rng.gen_range(0..n_distinct) as i64,
rng.gen_range(0..n_distinct) as i64,
rng.gen_range(0..n_distinct) as i64,
)
});
input4.iter_mut().for_each(|v| {
*v = rng.gen_range(0..n_distinct) as i64;
});
input123.sort();
let input1 = Int64Array::from_iter_values(input123.clone().into_iter().map(|k| k.0));
let input2 = Int64Array::from_iter_values(input123.clone().into_iter().map(|k| k.1));
let input3 = Int64Array::from_iter_values(input123.clone().into_iter().map(|k| k.2));
let input4 = Int64Array::from_iter_values(input4);
// split into several record batches
let mut remainder = RecordBatch::try_from_iter(vec![
("a", Arc::new(input1) as ArrayRef),
("b", Arc::new(input2) as ArrayRef),
("c", Arc::new(input3) as ArrayRef),
("d", Arc::new(input4) as ArrayRef),
])
.unwrap();
let mut batches = vec![];
if STREAM {
while remainder.num_rows() > 0 {
let batch_size = rng.gen_range(0..50);
if remainder.num_rows() < batch_size {
break;
}
batches.push(remainder.slice(0, batch_size));
remainder = remainder.slice(batch_size, remainder.num_rows() - batch_size);
}
} else {
while remainder.num_rows() > 0 {
let batch_size = rng.gen_range(0..remainder.num_rows() + 1);
batches.push(remainder.slice(0, batch_size));
remainder = remainder.slice(batch_size, remainder.num_rows() - batch_size);
}
}
add_empty_batches(batches, &mut rng)
}
/// Test group by with string/large string columns
#[tokio::test(flavor = "multi_thread")]
async fn group_by_strings() {
let mut join_set = JoinSet::new();
for large in [true, false] {
for sorted in [true, false] {
for generator in StringBatchGenerator::interesting_cases() {
join_set.spawn(group_by_string_test(generator, sorted, large));
}
}
}
while let Some(join_handle) = join_set.join_next().await {
// propagate errors
join_handle.unwrap();
}
}
/// Run GROUP BY <x> using SQL and ensure the results are correct
///
/// If sorted is true, the input batches will be sorted by the group by column
/// to test the streaming group by case
///
/// if large is true, the input batches will be LargeStringArray
async fn group_by_string_test(
mut generator: StringBatchGenerator,
sorted: bool,
large: bool,
) {
let column_name = "a";
let input = if sorted {
generator.make_sorted_input_batches(large)
} else {
generator.make_input_batches()
};
let expected = compute_counts(&input, column_name);
let schema = input[0].schema();
let session_config = SessionConfig::new().with_batch_size(50);
let ctx = SessionContext::new_with_config(session_config);
let provider = MemTable::try_new(schema.clone(), vec![input]).unwrap();
let provider = if sorted {
let sort_expr = datafusion::prelude::col("a").sort(true, true);
provider.with_sort_order(vec![vec![sort_expr]])
} else {
provider
};
ctx.register_table("t", Arc::new(provider)).unwrap();
let df = ctx
.sql("SELECT a, COUNT(*) FROM t GROUP BY a")
.await
.unwrap();
verify_ordered_aggregate(&df, sorted).await;
let results = df.collect().await.unwrap();
// verify that the results are correct
let actual = extract_result_counts(results);
assert_eq!(expected, actual);
}
async fn verify_ordered_aggregate(frame: &DataFrame, expected_sort: bool) {
struct Visitor {
expected_sort: bool,
}
let mut visitor = Visitor { expected_sort };
impl<'n> TreeNodeVisitor<'n> for Visitor {
type Node = Arc<dyn ExecutionPlan>;
fn f_down(&mut self, node: &'n Self::Node) -> Result<TreeNodeRecursion> {
if let Some(exec) = node.as_any().downcast_ref::<AggregateExec>() {
if self.expected_sort {
assert!(matches!(
exec.input_order_mode(),
InputOrderMode::PartiallySorted(_) | InputOrderMode::Sorted
));
} else {
assert!(matches!(exec.input_order_mode(), InputOrderMode::Linear));
}
}
Ok(TreeNodeRecursion::Continue)
}
}
let plan = frame.clone().create_physical_plan().await.unwrap();
plan.visit(&mut visitor).unwrap();
}
/// Compute the count of each distinct value in the specified column
///
/// ```text
/// +---------------+---------------+
/// | a | b |
/// +---------------+---------------+
/// | 𭏷𑩁 | 𘱦𫎛 |
/// | | 𬿪 |
/// ```
fn compute_counts(batches: &[RecordBatch], col: &str) -> HashMap<Option<String>, i64> {
let mut output = HashMap::new();
for arr in batches
.iter()
.map(|batch| batch.column_by_name(col).unwrap())
{
for value in to_str_vec(arr) {
output.entry(value).and_modify(|e| *e += 1).or_insert(1);
}
}
output
}
fn to_str_vec(array: &ArrayRef) -> Vec<Option<String>> {
match array.data_type() {
DataType::Utf8 => array
.as_string::<i32>()
.iter()
.map(|x| x.map(|x| x.to_string()))
.collect(),
DataType::LargeUtf8 => array
.as_string::<i64>()
.iter()
.map(|x| x.map(|x| x.to_string()))
.collect(),
_ => panic!("unexpected type"),
}
}
/// extracts the value of the first column and the count of the second column
/// ```text
/// +----------------+----------+
/// | a | COUNT(*) |
/// +----------------+----------+
/// | | 8 |
/// | | 11 |
/// ```
fn extract_result_counts(results: Vec<RecordBatch>) -> HashMap<Option<String>, i64> {
let group_arrays = results.iter().map(|batch| batch.column(0));
let count_arrays = results
.iter()
.map(|batch| batch.column(1).as_primitive::<Int64Type>());
let mut output = HashMap::new();
for (group_arr, count_arr) in group_arrays.zip(count_arrays) {
assert_eq!(group_arr.len(), count_arr.len());
let group_values = to_str_vec(group_arr);
for (group, count) in group_values.into_iter().zip(count_arr.iter()) {
assert!(output.get(&group).is_none());
let count = count.unwrap(); // counts can never be null
output.insert(group, count);
}
}
output
}