-
Notifications
You must be signed in to change notification settings - Fork 1.2k
/
row_hash.rs
799 lines (719 loc) · 34.8 KB
/
row_hash.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
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
// 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.
//! Hash aggregation
use std::sync::Arc;
use std::task::{Context, Poll};
use std::vec;
use crate::aggregates::group_values::{new_group_values, GroupValues};
use crate::aggregates::order::GroupOrderingFull;
use crate::aggregates::{
evaluate_group_by, evaluate_many, evaluate_optional, group_schema, AggregateMode,
PhysicalGroupBy,
};
use crate::common::IPCWriter;
use crate::metrics::{BaselineMetrics, RecordOutput};
use crate::sorts::sort::{read_spill_as_stream, sort_batch};
use crate::sorts::streaming_merge;
use crate::stream::RecordBatchStreamAdapter;
use crate::{aggregates, ExecutionPlan, PhysicalExpr};
use crate::{RecordBatchStream, SendableRecordBatchStream};
use arrow::array::*;
use arrow::{datatypes::SchemaRef, record_batch::RecordBatch};
use arrow_schema::SortOptions;
use datafusion_common::{DataFusionError, Result};
use datafusion_execution::disk_manager::RefCountedTempFile;
use datafusion_execution::memory_pool::proxy::VecAllocExt;
use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation};
use datafusion_execution::runtime_env::RuntimeEnv;
use datafusion_execution::TaskContext;
use datafusion_expr::{EmitTo, GroupsAccumulator};
use datafusion_physical_expr::expressions::Column;
use datafusion_physical_expr::{
AggregateExpr, GroupsAccumulatorAdapter, PhysicalSortExpr,
};
use futures::ready;
use futures::stream::{Stream, StreamExt};
use log::debug;
#[derive(Debug, Clone)]
/// This object tracks the aggregation phase (input/output)
pub(crate) enum ExecutionState {
ReadingInput,
/// When producing output, the remaining rows to output are stored
/// here and are sliced off as needed in batch_size chunks
ProducingOutput(RecordBatch),
Done,
}
use super::order::GroupOrdering;
use super::AggregateExec;
/// This encapsulates the spilling state
struct SpillState {
/// If data has previously been spilled, the locations of the
/// spill files (in Arrow IPC format)
spills: Vec<RefCountedTempFile>,
/// Sorting expression for spilling batches
spill_expr: Vec<PhysicalSortExpr>,
/// Schema for spilling batches
spill_schema: SchemaRef,
/// true when streaming merge is in progress
is_stream_merging: bool,
/// aggregate_arguments for merging spilled data
merging_aggregate_arguments: Vec<Vec<Arc<dyn PhysicalExpr>>>,
/// GROUP BY expressions for merging spilled data
merging_group_by: PhysicalGroupBy,
}
/// HashTable based Grouping Aggregator
///
/// # Design Goals
///
/// This structure is designed so that updating the aggregates can be
/// vectorized (done in a tight loop) without allocations. The
/// accumulator state is *not* managed by this operator (e.g in the
/// hash table) and instead is delegated to the individual
/// accumulators which have type specialized inner loops that perform
/// the aggregation.
///
/// # Architecture
///
/// ```text
///
/// Assigns a consecutive group internally stores aggregate values
/// index for each unique set for all groups
/// of group values
///
/// ┌────────────┐ ┌──────────────┐ ┌──────────────┐
/// │ ┌────────┐ │ │┌────────────┐│ │┌────────────┐│
/// │ │ "A" │ │ ││accumulator ││ ││accumulator ││
/// │ ├────────┤ │ ││ 0 ││ ││ N ││
/// │ │ "Z" │ │ ││ ┌────────┐ ││ ││ ┌────────┐ ││
/// │ └────────┘ │ ││ │ state │ ││ ││ │ state │ ││
/// │ │ ││ │┌─────┐ │ ││ ... ││ │┌─────┐ │ ││
/// │ ... │ ││ │├─────┤ │ ││ ││ │├─────┤ │ ││
/// │ │ ││ │└─────┘ │ ││ ││ │└─────┘ │ ││
/// │ │ ││ │ │ ││ ││ │ │ ││
/// │ ┌────────┐ │ ││ │ ... │ ││ ││ │ ... │ ││
/// │ │ "Q" │ │ ││ │ │ ││ ││ │ │ ││
/// │ └────────┘ │ ││ │┌─────┐ │ ││ ││ │┌─────┐ │ ││
/// │ │ ││ │└─────┘ │ ││ ││ │└─────┘ │ ││
/// └────────────┘ ││ └────────┘ ││ ││ └────────┘ ││
/// │└────────────┘│ │└────────────┘│
/// └──────────────┘ └──────────────┘
///
/// group_values accumulators
///
/// ```
///
/// For example, given a query like `COUNT(x), SUM(y) ... GROUP BY z`,
/// [`group_values`] will store the distinct values of `z`. There will
/// be one accumulator for `COUNT(x)`, specialized for the data type
/// of `x` and one accumulator for `SUM(y)`, specialized for the data
/// type of `y`.
///
/// # Description
///
/// [`group_values`] does not store any aggregate state inline. It only
/// assigns "group indices", one for each (distinct) group value. The
/// accumulators manage the in-progress aggregate state for each
/// group, with the group values themselves are stored in
/// [`group_values`] at the corresponding group index.
///
/// The accumulator state (e.g partial sums) is managed by and stored
/// by a [`GroupsAccumulator`] accumulator. There is one accumulator
/// per aggregate expression (COUNT, AVG, etc) in the
/// stream. Internally, each `GroupsAccumulator` manages the state for
/// multiple groups, and is passed `group_indexes` during update. Note
/// The accumulator state is not managed by this operator (e.g in the
/// hash table).
///
/// [`group_values`]: Self::group_values
///
/// # Spilling
///
/// The sizes of group values and accumulators can become large. Before that causes out of memory,
/// this hash aggregator outputs partial states early for partial aggregation or spills to local
/// disk using Arrow IPC format for final aggregation. For every input [`RecordBatch`], the memory
/// manager checks whether the new input size meets the memory configuration. If not, outputting or
/// spilling happens. For outputting, the final aggregation takes care of re-grouping. For spilling,
/// later stream-merge sort on reading back the spilled data does re-grouping. Note the rows cannot
/// be grouped once spilled onto disk, the read back data needs to be re-grouped again. In addition,
/// re-grouping may cause out of memory again. Thus, re-grouping has to be a sort based aggregation.
///
/// ```text
/// Partial Aggregation [batch_size = 2] (max memory = 3 rows)
///
/// INPUTS PARTIALLY AGGREGATED (UPDATE BATCH) OUTPUTS
/// ┌─────────┐ ┌─────────────────┐ ┌─────────────────┐
/// │ a │ b │ │ a │ AVG(b) │ │ a │ AVG(b) │
/// │---│-----│ │ │[count]│[sum]│ │ │[count]│[sum]│
/// │ 3 │ 3.0 │ ─▶ │---│-------│-----│ │---│-------│-----│
/// │ 2 │ 2.0 │ │ 2 │ 1 │ 2.0 │ ─▶ early emit ─▶ │ 2 │ 1 │ 2.0 │
/// └─────────┘ │ 3 │ 2 │ 7.0 │ │ │ 3 │ 2 │ 7.0 │
/// ┌─────────┐ ─▶ │ 4 │ 1 │ 8.0 │ │ └─────────────────┘
/// │ 3 │ 4.0 │ └─────────────────┘ └▶ ┌─────────────────┐
/// │ 4 │ 8.0 │ ┌─────────────────┐ │ 4 │ 1 │ 8.0 │
/// └─────────┘ │ a │ AVG(b) │ ┌▶ │ 1 │ 1 │ 1.0 │
/// ┌─────────┐ │---│-------│-----│ │ └─────────────────┘
/// │ 1 │ 1.0 │ ─▶ │ 1 │ 1 │ 1.0 │ ─▶ early emit ─▶ ┌─────────────────┐
/// │ 3 │ 2.0 │ │ 3 │ 1 │ 2.0 │ │ 3 │ 1 │ 2.0 │
/// └─────────┘ └─────────────────┘ └─────────────────┘
///
///
/// Final Aggregation [batch_size = 2] (max memory = 3 rows)
///
/// PARTIALLY INPUTS FINAL AGGREGATION (MERGE BATCH) RE-GROUPED (SORTED)
/// ┌─────────────────┐ [keep using the partial schema] [Real final aggregation
/// │ a │ AVG(b) │ ┌─────────────────┐ output]
/// │ │[count]│[sum]│ │ a │ AVG(b) │ ┌────────────┐
/// │---│-------│-----│ ─▶ │ │[count]│[sum]│ │ a │ AVG(b) │
/// │ 3 │ 3 │ 3.0 │ │---│-------│-----│ ─▶ spill ─┐ │---│--------│
/// │ 2 │ 2 │ 1.0 │ │ 2 │ 2 │ 1.0 │ │ │ 1 │ 4.0 │
/// └─────────────────┘ │ 3 │ 4 │ 8.0 │ ▼ │ 2 │ 1.0 │
/// ┌─────────────────┐ ─▶ │ 4 │ 1 │ 7.0 │ Streaming ─▶ └────────────┘
/// │ 3 │ 1 │ 5.0 │ └─────────────────┘ merge sort ─▶ ┌────────────┐
/// │ 4 │ 1 │ 7.0 │ ┌─────────────────┐ ▲ │ a │ AVG(b) │
/// └─────────────────┘ │ a │ AVG(b) │ │ │---│--------│
/// ┌─────────────────┐ │---│-------│-----│ ─▶ memory ─┘ │ 3 │ 2.0 │
/// │ 1 │ 2 │ 8.0 │ ─▶ │ 1 │ 2 │ 8.0 │ │ 4 │ 7.0 │
/// │ 2 │ 2 │ 3.0 │ │ 2 │ 2 │ 3.0 │ └────────────┘
/// └─────────────────┘ └─────────────────┘
/// ```
pub(crate) struct GroupedHashAggregateStream {
schema: SchemaRef,
input: SendableRecordBatchStream,
mode: AggregateMode,
/// Accumulators, one for each `AggregateExpr` in the query
///
/// For example, if the query has aggregates, `SUM(x)`,
/// `COUNT(y)`, there will be two accumulators, each one
/// specialized for that particular aggregate and its input types
accumulators: Vec<Box<dyn GroupsAccumulator>>,
/// Arguments to pass to each accumulator.
///
/// The arguments in `accumulator[i]` is passed `aggregate_arguments[i]`
///
/// The argument to each accumulator is itself a `Vec` because
/// some aggregates such as `CORR` can accept more than one
/// argument.
aggregate_arguments: Vec<Vec<Arc<dyn PhysicalExpr>>>,
/// Optional filter expression to evaluate, one for each for
/// accumulator. If present, only those rows for which the filter
/// evaluate to true should be included in the aggregate results.
///
/// For example, for an aggregate like `SUM(x) FILTER (WHERE x >= 100)`,
/// the filter expression is `x > 100`.
filter_expressions: Vec<Option<Arc<dyn PhysicalExpr>>>,
/// GROUP BY expressions
group_by: PhysicalGroupBy,
/// The memory reservation for this grouping
reservation: MemoryReservation,
/// An interning store of group keys
group_values: Box<dyn GroupValues>,
/// scratch space for the current input [`RecordBatch`] being
/// processed. Reused across batches here to avoid reallocations
current_group_indices: Vec<usize>,
/// Tracks if this stream is generating input or output
exec_state: ExecutionState,
/// Execution metrics
baseline_metrics: BaselineMetrics,
/// max rows in output RecordBatches
batch_size: usize,
/// Optional ordering information, that might allow groups to be
/// emitted from the hash table prior to seeing the end of the
/// input
group_ordering: GroupOrdering,
/// Have we seen the end of the input
input_done: bool,
/// The [`RuntimeEnv`] associated with the [`TaskContext`] argument
runtime: Arc<RuntimeEnv>,
/// The spill state object
spill_state: SpillState,
/// Optional soft limit on the number of `group_values` in a batch
/// If the number of `group_values` in a single batch exceeds this value,
/// the `GroupedHashAggregateStream` operation immediately switches to
/// output mode and emits all groups.
group_values_soft_limit: Option<usize>,
}
impl GroupedHashAggregateStream {
/// Create a new GroupedHashAggregateStream
pub fn new(
agg: &AggregateExec,
context: Arc<TaskContext>,
partition: usize,
) -> Result<Self> {
debug!("Creating GroupedHashAggregateStream");
let agg_schema = Arc::clone(&agg.schema);
let agg_group_by = agg.group_by.clone();
let agg_filter_expr = agg.filter_expr.clone();
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 timer = baseline_metrics.elapsed_compute().timer();
let aggregate_exprs = agg.aggr_expr.clone();
// arguments for each aggregate, one vec of expressions per
// aggregate
let aggregate_arguments = aggregates::aggregate_expressions(
&agg.aggr_expr,
&agg.mode,
agg_group_by.expr.len(),
)?;
// arguments for aggregating spilled data is the same as the one for final aggregation
let merging_aggregate_arguments = aggregates::aggregate_expressions(
&agg.aggr_expr,
&AggregateMode::Final,
agg_group_by.expr.len(),
)?;
let filter_expressions = match agg.mode {
AggregateMode::Partial
| AggregateMode::Single
| AggregateMode::SinglePartitioned => agg_filter_expr,
AggregateMode::Final | AggregateMode::FinalPartitioned => {
vec![None; agg.aggr_expr.len()]
}
};
// Instantiate the accumulators
let accumulators: Vec<_> = aggregate_exprs
.iter()
.map(create_group_accumulator)
.collect::<Result<_>>()?;
let group_schema = group_schema(&agg_schema, agg_group_by.expr.len());
let spill_expr = group_schema
.fields
.into_iter()
.enumerate()
.map(|(idx, field)| PhysicalSortExpr {
expr: Arc::new(Column::new(field.name().as_str(), idx)) as _,
options: SortOptions::default(),
})
.collect();
let name = format!("GroupedHashAggregateStream[{partition}]");
let reservation = MemoryConsumer::new(name)
.with_can_spill(true)
.register(context.memory_pool());
let (ordering, _) = agg
.properties()
.equivalence_properties()
.find_longest_permutation(&agg_group_by.output_exprs());
let group_ordering = GroupOrdering::try_new(
&group_schema,
&agg.input_order_mode,
ordering.as_slice(),
)?;
let group_values = new_group_values(group_schema)?;
timer.done();
let exec_state = ExecutionState::ReadingInput;
let spill_state = SpillState {
spills: vec![],
spill_expr,
spill_schema: agg_schema.clone(),
is_stream_merging: false,
merging_aggregate_arguments,
merging_group_by: PhysicalGroupBy::new_single(agg_group_by.expr.clone()),
};
Ok(GroupedHashAggregateStream {
schema: agg_schema,
input,
mode: agg.mode,
accumulators,
aggregate_arguments,
filter_expressions,
group_by: agg_group_by,
reservation,
group_values,
current_group_indices: Default::default(),
exec_state,
baseline_metrics,
batch_size,
group_ordering,
input_done: false,
runtime: context.runtime_env(),
spill_state,
group_values_soft_limit: agg.limit,
})
}
}
/// Create an accumulator for `agg_expr` -- a [`GroupsAccumulator`] if
/// that is supported by the aggregate, or a
/// [`GroupsAccumulatorAdapter`] if not.
pub(crate) fn create_group_accumulator(
agg_expr: &Arc<dyn AggregateExpr>,
) -> Result<Box<dyn GroupsAccumulator>> {
if agg_expr.groups_accumulator_supported() {
agg_expr.create_groups_accumulator()
} else {
// Note in the log when the slow path is used
debug!(
"Creating GroupsAccumulatorAdapter for {}: {agg_expr:?}",
agg_expr.name()
);
let agg_expr_captured = agg_expr.clone();
let factory = move || agg_expr_captured.create_accumulator();
Ok(Box::new(GroupsAccumulatorAdapter::new(factory)))
}
}
/// Extracts a successful Ok(_) or returns Poll::Ready(Some(Err(e))) with errors
macro_rules! extract_ok {
($RES: expr) => {{
match $RES {
Ok(v) => v,
Err(e) => return Poll::Ready(Some(Err(e))),
}
}};
}
impl Stream for GroupedHashAggregateStream {
type Item = Result<RecordBatch>;
fn poll_next(
mut self: std::pin::Pin<&mut Self>,
cx: &mut Context<'_>,
) -> Poll<Option<Self::Item>> {
let elapsed_compute = self.baseline_metrics.elapsed_compute().clone();
loop {
match &self.exec_state {
ExecutionState::ReadingInput => 'reading_input: {
match ready!(self.input.poll_next_unpin(cx)) {
// new batch to aggregate
Some(Ok(batch)) => {
let timer = elapsed_compute.timer();
// Make sure we have enough capacity for `batch`, otherwise spill
extract_ok!(self.spill_previous_if_necessary(&batch));
// Do the grouping
extract_ok!(self.group_aggregate_batch(batch));
// If we can begin emitting rows, do so,
// otherwise keep consuming input
assert!(!self.input_done);
// If the number of group values equals or exceeds the soft limit,
// emit all groups and switch to producing output
if self.hit_soft_group_limit() {
timer.done();
extract_ok!(self.set_input_done_and_produce_output());
// make sure the exec_state just set is not overwritten below
break 'reading_input;
}
if let Some(to_emit) = self.group_ordering.emit_to() {
let batch = extract_ok!(self.emit(to_emit, false));
self.exec_state = ExecutionState::ProducingOutput(batch);
timer.done();
// make sure the exec_state just set is not overwritten below
break 'reading_input;
}
extract_ok!(self.emit_early_if_necessary());
timer.done();
}
Some(Err(e)) => {
// inner had error, return to caller
return Poll::Ready(Some(Err(e)));
}
None => {
// inner is done, emit all rows and switch to producing output
extract_ok!(self.set_input_done_and_produce_output());
}
}
}
ExecutionState::ProducingOutput(batch) => {
// slice off a part of the batch, if needed
let output_batch;
let size = self.batch_size;
(self.exec_state, output_batch) = if batch.num_rows() <= size {
(
if self.input_done {
ExecutionState::Done
} else {
ExecutionState::ReadingInput
},
batch.clone(),
)
} else {
// output first batch_size rows
let size = self.batch_size;
let num_remaining = batch.num_rows() - size;
let remaining = batch.slice(size, num_remaining);
let output = batch.slice(0, size);
(ExecutionState::ProducingOutput(remaining), output)
};
return Poll::Ready(Some(Ok(
output_batch.record_output(&self.baseline_metrics)
)));
}
ExecutionState::Done => {
// release the memory reservation since sending back output batch itself needs
// some memory reservation, so make some room for it.
self.clear_all();
let _ = self.update_memory_reservation();
return Poll::Ready(None);
}
}
}
}
}
impl RecordBatchStream for GroupedHashAggregateStream {
fn schema(&self) -> SchemaRef {
self.schema.clone()
}
}
impl GroupedHashAggregateStream {
/// Perform group-by aggregation for the given [`RecordBatch`].
fn group_aggregate_batch(&mut self, batch: RecordBatch) -> Result<()> {
// Evaluate the grouping expressions
let group_by_values = if self.spill_state.is_stream_merging {
evaluate_group_by(&self.spill_state.merging_group_by, &batch)?
} else {
evaluate_group_by(&self.group_by, &batch)?
};
// 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)?
};
// Evaluate the filter expressions, if any, against the inputs
let filter_values = if self.spill_state.is_stream_merging {
let filter_expressions = vec![None; self.accumulators.len()];
evaluate_optional(&filter_expressions, &batch)?
} else {
evaluate_optional(&self.filter_expressions, &batch)?
};
for group_values in &group_by_values {
// calculate the group indices for each input row
let starting_num_groups = self.group_values.len();
self.group_values
.intern(group_values, &mut self.current_group_indices)?;
let group_indices = &self.current_group_indices;
// Update ordering information if necessary
let total_num_groups = self.group_values.len();
if total_num_groups > starting_num_groups {
self.group_ordering.new_groups(
group_values,
group_indices,
total_num_groups,
)?;
}
// Gather the inputs to call the actual accumulator
let t = self
.accumulators
.iter_mut()
.zip(input_values.iter())
.zip(filter_values.iter());
for ((acc, values), opt_filter) in t {
let opt_filter = opt_filter.as_ref().map(|filter| filter.as_boolean());
// Call the appropriate method on each aggregator with
// the entire input row and the relevant group indexes
match self.mode {
AggregateMode::Partial
| AggregateMode::Single
| AggregateMode::SinglePartitioned
if !self.spill_state.is_stream_merging =>
{
acc.update_batch(
values,
group_indices,
opt_filter,
total_num_groups,
)?;
}
_ => {
// if aggregation is over intermediate states,
// use merge
acc.merge_batch(
values,
group_indices,
opt_filter,
total_num_groups,
)?;
}
}
}
}
match self.update_memory_reservation() {
// Here we can ignore `insufficient_capacity_err` because we will spill later,
// but at least one batch should fit in the memory
Err(DataFusionError::ResourcesExhausted(_))
if self.group_values.len() >= self.batch_size =>
{
Ok(())
}
other => other,
}
}
fn update_memory_reservation(&mut self) -> Result<()> {
let acc = self.accumulators.iter().map(|x| x.size()).sum::<usize>();
self.reservation.try_resize(
acc + self.group_values.size()
+ self.group_ordering.size()
+ self.current_group_indices.allocated_size(),
)
}
/// Create an output RecordBatch with the group keys and
/// accumulator states/values specified in emit_to
fn emit(&mut self, emit_to: EmitTo, spilling: bool) -> Result<RecordBatch> {
let schema = if spilling {
self.spill_state.spill_schema.clone()
} else {
self.schema()
};
if self.group_values.is_empty() {
return Ok(RecordBatch::new_empty(schema));
}
let mut output = self.group_values.emit(emit_to)?;
if let EmitTo::First(n) = emit_to {
self.group_ordering.remove_groups(n);
}
// Next output each aggregate value
for acc in self.accumulators.iter_mut() {
match self.mode {
AggregateMode::Partial => output.extend(acc.state(emit_to)?),
_ if spilling => {
// If spilling, output partial state because the spilled data will be
// merged and re-evaluated later.
output.extend(acc.state(emit_to)?)
}
AggregateMode::Final
| AggregateMode::FinalPartitioned
| AggregateMode::Single
| AggregateMode::SinglePartitioned => output.push(acc.evaluate(emit_to)?),
}
}
// 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)?;
Ok(batch)
}
/// Optimistically, [`Self::group_aggregate_batch`] allows to exceed the memory target slightly
/// (~ 1 [`RecordBatch`]) for simplicity. In such cases, spill the data to disk and clear the
/// memory. Currently only [`GroupOrdering::None`] is supported for spilling.
fn spill_previous_if_necessary(&mut self, batch: &RecordBatch) -> Result<()> {
// TODO: support group_ordering for spilling
if self.group_values.len() > 0
&& batch.num_rows() > 0
&& matches!(self.group_ordering, GroupOrdering::None)
&& !matches!(self.mode, AggregateMode::Partial)
&& !self.spill_state.is_stream_merging
&& self.update_memory_reservation().is_err()
{
// Use input batch (Partial mode) schema for spilling because
// the spilled data will be merged and re-evaluated later.
self.spill_state.spill_schema = batch.schema();
self.spill()?;
self.clear_shrink(batch);
}
Ok(())
}
/// Emit all rows, sort them, and store them on disk.
fn spill(&mut self) -> Result<()> {
let emit = self.emit(EmitTo::All, true)?;
let sorted = sort_batch(&emit, &self.spill_state.spill_expr, None)?;
let spillfile = self.runtime.disk_manager.create_tmp_file("HashAggSpill")?;
let mut writer = IPCWriter::new(spillfile.path(), &emit.schema())?;
// TODO: slice large `sorted` and write to multiple files in parallel
let mut offset = 0;
let total_rows = sorted.num_rows();
while offset < total_rows {
let length = std::cmp::min(total_rows - offset, self.batch_size);
let batch = sorted.slice(offset, length);
offset += batch.num_rows();
writer.write(&batch)?;
}
writer.finish()?;
self.spill_state.spills.push(spillfile);
Ok(())
}
/// Clear memory and shirk capacities to the size of the batch.
fn clear_shrink(&mut self, batch: &RecordBatch) {
self.group_values.clear_shrink(batch);
self.current_group_indices.clear();
self.current_group_indices.shrink_to(batch.num_rows());
}
/// Clear memory and shirk capacities to zero.
fn clear_all(&mut self) {
let s = self.schema();
self.clear_shrink(&RecordBatch::new_empty(s));
}
/// Emit if the used memory exceeds the target for partial aggregation.
/// Currently only [`GroupOrdering::None`] is supported for early emitting.
/// TODO: support group_ordering for early emitting
fn emit_early_if_necessary(&mut self) -> Result<()> {
if self.group_values.len() >= self.batch_size
&& matches!(self.group_ordering, GroupOrdering::None)
&& matches!(self.mode, AggregateMode::Partial)
&& self.update_memory_reservation().is_err()
{
let n = self.group_values.len() / self.batch_size * self.batch_size;
let batch = self.emit(EmitTo::First(n), false)?;
self.exec_state = ExecutionState::ProducingOutput(batch);
}
Ok(())
}
/// At this point, all the inputs are read and there are some spills.
/// Emit the remaining rows and create a batch.
/// Conduct a streaming merge sort between the batch and spilled data. Since the stream is fully
/// sorted, set `self.group_ordering` to Full, then later we can read with [`EmitTo::First`].
fn update_merged_stream(&mut self) -> Result<()> {
let batch = self.emit(EmitTo::All, true)?;
// clear up memory for streaming_merge
self.clear_all();
self.update_memory_reservation()?;
let mut streams: Vec<SendableRecordBatchStream> = vec![];
let expr = self.spill_state.spill_expr.clone();
let schema = batch.schema();
streams.push(Box::pin(RecordBatchStreamAdapter::new(
schema.clone(),
futures::stream::once(futures::future::lazy(move |_| {
sort_batch(&batch, &expr, None)
})),
)));
for spill in self.spill_state.spills.drain(..) {
let stream = read_spill_as_stream(spill, schema.clone())?;
streams.push(stream);
}
self.spill_state.is_stream_merging = true;
self.input = streaming_merge(
streams,
schema,
&self.spill_state.spill_expr,
self.baseline_metrics.clone(),
self.batch_size,
None,
self.reservation.new_empty(),
)?;
self.input_done = false;
self.group_ordering = GroupOrdering::Full(GroupOrderingFull::new());
Ok(())
}
/// returns true if there is a soft groups limit and the number of distinct
/// groups we have seen is over that limit
fn hit_soft_group_limit(&self) -> bool {
let Some(group_values_soft_limit) = self.group_values_soft_limit else {
return false;
};
group_values_soft_limit <= self.group_values.len()
}
/// common function for signalling end of processing of the input stream
fn set_input_done_and_produce_output(&mut self) -> Result<()> {
self.input_done = true;
self.group_ordering.input_done();
let elapsed_compute = self.baseline_metrics.elapsed_compute().clone();
let timer = elapsed_compute.timer();
self.exec_state = if self.spill_state.spills.is_empty() {
let batch = self.emit(EmitTo::All, false)?;
ExecutionState::ProducingOutput(batch)
} else {
// If spill files exist, stream-merge them.
self.update_merged_stream()?;
ExecutionState::ReadingInput
};
timer.done();
Ok(())
}
}