forked from apache/datafusion
-
Notifications
You must be signed in to change notification settings - Fork 5
/
Copy pathhash_aggregate.rs
1981 lines (1813 loc) · 70.3 KB
/
hash_aggregate.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
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
// 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.
//! Defines the execution plan for the hash aggregate operation
use std::any::Any;
use std::sync::Arc;
use std::task::{Context, Poll};
use std::vec;
use ahash::RandomState;
use futures::{
stream::{Stream, StreamExt},
Future,
};
use crate::cube_match_scalar;
use crate::error::{DataFusionError, Result};
use crate::physical_plan::{
Accumulator, AggregateExpr, DisplayFormatType, Distribution, ExecutionPlan,
OptimizerHints, Partitioning, PhysicalExpr, SQLMetric,
};
use crate::scalar::ScalarValue;
use arrow::{
array::{Array, UInt32Builder},
error::{ArrowError, Result as ArrowResult},
};
use arrow::{
array::{
ArrayRef, Float32Array, Float64Array, Int16Array, Int32Array, Int64Array,
Int64Decimal0Array, Int64Decimal10Array, Int64Decimal1Array, Int64Decimal2Array,
Int64Decimal3Array, Int64Decimal4Array, Int64Decimal5Array, Int8Array,
Int96Array, Int96Decimal0Array, Int96Decimal10Array, Int96Decimal1Array,
Int96Decimal2Array, Int96Decimal3Array, Int96Decimal4Array, Int96Decimal5Array,
StringArray, UInt16Array, UInt32Array, UInt64Array, UInt8Array,
},
compute,
};
use arrow::{
array::{BooleanArray, Date32Array, DictionaryArray},
datatypes::{
ArrowDictionaryKeyType, ArrowNativeType, Int16Type, Int32Type, Int64Type,
Int8Type, UInt16Type, UInt32Type, UInt64Type, UInt8Type,
},
};
use arrow::{
datatypes::{DataType, Field, Schema, SchemaRef, TimeUnit},
record_batch::RecordBatch,
};
use hashbrown::HashMap;
use pin_project_lite::pin_project;
use arrow::array::{
ArrayBuilder, BinaryBuilder, LargeStringArray, StringBuilder,
TimestampMicrosecondArray, TimestampMillisecondArray, TimestampNanosecondArray,
};
use async_trait::async_trait;
use super::groups_accumulator::{EmitTo, GroupsAccumulator};
use super::groups_accumulator_flat_adapter::GroupsAccumulatorFlatAdapter;
use super::{
expressions::Column, group_scalar::GroupByScalar, RecordBatchStream,
SendableRecordBatchStream,
};
use crate::cube_ext;
use crate::cube_ext::ordfloat::{OrdF32, OrdF64};
use crate::physical_plan::sorted_aggregate::SortedAggState;
use compute::cast;
use smallvec::smallvec;
use smallvec::SmallVec;
use std::convert::TryFrom;
/// Hash aggregate modes
#[derive(Debug, Copy, Clone, PartialEq, Eq)]
pub enum AggregateMode {
/// Partial aggregate that can be applied in parallel across input partitions
Partial,
/// Final aggregate that produces a single partition of output
Final,
/// Combines `Partial` and `Final` in a single pass. Saves time, but not always possible.
Full,
/// Final aggregate that works on pre-partitioned data.
///
/// This requires the invariant that all rows with a particular
/// grouping key are in the same partitions, such as is the case
/// with Hash repartitioning on the group keys. If a group key is
/// duplicated, duplicate groups would be produced
FinalPartitioned,
}
/// Defines which aggregation algorithm is used
#[derive(Debug, Copy, Clone, PartialEq, Eq)]
pub enum AggregateStrategy {
/// Build a hash map with accumulators. General-purpose
Hash,
/// Aggregate group on-the-fly for sorted inputs. Faster than hash, but requires sorted input
InplaceSorted,
}
/// Hash aggregate execution plan
#[derive(Debug)]
pub struct HashAggregateExec {
strategy: AggregateStrategy,
output_sort_order: Option<Vec<usize>>,
/// Aggregation mode (full, partial)
mode: AggregateMode,
/// Grouping expressions
group_expr: Vec<(Arc<dyn PhysicalExpr>, String)>,
/// Aggregate expressions
aggr_expr: Vec<Arc<dyn AggregateExpr>>,
/// Input plan, could be a partial aggregate or the input to the aggregate
input: Arc<dyn ExecutionPlan>,
/// Schema after the aggregate is applied
schema: SchemaRef,
/// Input schema before any aggregation is applied. For partial aggregate this will be the
/// same as input.schema() but for the final aggregate it will be the same as the input
/// to the partial aggregate
input_schema: SchemaRef,
/// Metric to track number of output rows
output_rows: Arc<SQLMetric>,
}
pub(crate) fn create_schema(
input_schema: &Schema,
group_expr: &[(Arc<dyn PhysicalExpr>, String)],
aggr_expr: &[Arc<dyn AggregateExpr>],
mode: AggregateMode,
) -> Result<Schema> {
let mut fields = Vec::with_capacity(group_expr.len() + aggr_expr.len());
for (expr, name) in group_expr {
fields.push(Field::new(
name,
expr.data_type(input_schema)?,
expr.nullable(input_schema)?,
))
}
match mode {
AggregateMode::Partial => {
// in partial mode, the fields of the accumulator's state
for expr in aggr_expr {
fields.extend(expr.state_fields()?.iter().cloned())
}
}
AggregateMode::Final | AggregateMode::Full | AggregateMode::FinalPartitioned => {
// in final mode, the field with the final result of the accumulator
for expr in aggr_expr {
fields.push(expr.field()?)
}
}
}
Ok(Schema::new(fields))
}
impl HashAggregateExec {
/// Create a new hash aggregate execution plan
pub fn try_new(
strategy: AggregateStrategy,
output_sort_order: Option<Vec<usize>>,
mode: AggregateMode,
group_expr: Vec<(Arc<dyn PhysicalExpr>, String)>,
aggr_expr: Vec<Arc<dyn AggregateExpr>>,
input: Arc<dyn ExecutionPlan>,
input_schema: SchemaRef,
) -> Result<Self> {
let schema = create_schema(&input.schema(), &group_expr, &aggr_expr, mode)?;
let schema = Arc::new(schema);
let output_rows = SQLMetric::counter();
match strategy {
AggregateStrategy::Hash => assert!(output_sort_order.is_none()),
AggregateStrategy::InplaceSorted => {
assert!(output_sort_order.is_some());
assert!(
output_sort_order
.as_ref()
.unwrap()
.iter()
.all(|i| *i < group_expr.len()),
"sort_order mentions value columns"
);
}
}
Ok(HashAggregateExec {
strategy,
output_sort_order,
mode,
group_expr,
aggr_expr,
input,
schema,
input_schema,
output_rows,
})
}
/// Aggregation strategy.
pub fn strategy(&self) -> AggregateStrategy {
self.strategy
}
/// Aggregation mode (full, partial)
pub fn mode(&self) -> &AggregateMode {
&self.mode
}
/// Grouping expressions
pub fn group_expr(&self) -> &[(Arc<dyn PhysicalExpr>, String)] {
&self.group_expr
}
/// Aggregate expressions
pub fn aggr_expr(&self) -> &[Arc<dyn AggregateExpr>] {
&self.aggr_expr
}
/// Input plan
pub fn input(&self) -> &Arc<dyn ExecutionPlan> {
&self.input
}
/// Get the input schema before any aggregates are applied
pub fn input_schema(&self) -> SchemaRef {
self.input_schema.clone()
}
}
#[async_trait]
impl ExecutionPlan for HashAggregateExec {
/// Return a reference to Any that can be used for downcasting
fn as_any(&self) -> &dyn Any {
self
}
fn schema(&self) -> SchemaRef {
self.schema.clone()
}
fn children(&self) -> Vec<Arc<dyn ExecutionPlan>> {
vec![self.input.clone()]
}
fn required_child_distribution(&self) -> Distribution {
match &self.mode {
AggregateMode::Partial | AggregateMode::Full => {
Distribution::UnspecifiedDistribution
}
AggregateMode::FinalPartitioned => Distribution::HashPartitioned(
self.group_expr.iter().map(|x| x.0.clone()).collect(),
),
AggregateMode::Final => Distribution::SinglePartition,
}
}
/// Get the output partitioning of this plan
fn output_partitioning(&self) -> Partitioning {
self.input.output_partitioning()
}
async fn execute(&self, partition: usize) -> Result<SendableRecordBatchStream> {
let input = self.input.execute(partition).await?;
let group_expr = self.group_expr.iter().map(|x| x.0.clone()).collect();
if self.group_expr.is_empty() {
Ok(Box::pin(HashAggregateStream::new(
self.mode,
self.schema.clone(),
self.aggr_expr.clone(),
input,
)))
} else {
Ok(Box::pin(GroupedHashAggregateStream::new(
self.strategy,
self.mode,
self.schema.clone(),
group_expr,
self.aggr_expr.clone(),
input,
self.output_rows.clone(),
)))
}
}
fn with_new_children(
&self,
children: Vec<Arc<dyn ExecutionPlan>>,
) -> Result<Arc<dyn ExecutionPlan>> {
match children.len() {
1 => Ok(Arc::new(HashAggregateExec::try_new(
self.strategy,
self.output_sort_order.clone(),
self.mode,
self.group_expr.clone(),
self.aggr_expr.clone(),
children[0].clone(),
self.input_schema.clone(),
)?)),
_ => Err(DataFusionError::Internal(
"HashAggregateExec wrong number of children".to_string(),
)),
}
}
fn output_hints(&self) -> OptimizerHints {
let sort_order = match self.strategy {
AggregateStrategy::Hash => None,
AggregateStrategy::InplaceSorted => self.output_sort_order.clone(),
};
OptimizerHints {
sort_order,
single_value_columns: Vec::new(),
}
}
fn metrics(&self) -> HashMap<String, SQLMetric> {
let mut metrics = HashMap::new();
metrics.insert("outputRows".to_owned(), (*self.output_rows).clone());
metrics
}
fn fmt_as(
&self,
t: DisplayFormatType,
f: &mut std::fmt::Formatter,
) -> std::fmt::Result {
match t {
DisplayFormatType::Default => {
write!(f, "HashAggregateExec: mode={:?}", self.mode)?;
let g: Vec<String> = self
.group_expr
.iter()
.map(|(e, alias)| {
let e = e.to_string();
if &e != alias {
format!("{} as {}", e, alias)
} else {
e
}
})
.collect();
write!(f, ", gby=[{}]", g.join(", "))?;
let a: Vec<String> = self
.aggr_expr
.iter()
.map(|agg| agg.name().to_string())
.collect();
write!(f, ", aggr=[{}]", a.join(", "))?;
}
}
Ok(())
}
}
/*
The architecture is the following:
1. An accumulator has state that is updated on each batch.
2. At the end of the aggregation (e.g. end of batches in a partition), the accumulator converts its state to a RecordBatch of a single row
3. The RecordBatches of all accumulators are merged (`concatenate` in `rust/arrow`) together to a single RecordBatch.
4. The state's RecordBatch is `merge`d to a new state
5. The state is mapped to the final value
Why:
* Accumulators' state can be statically typed, but it is more efficient to transmit data from the accumulators via `Array`
* The `merge` operation must have access to the state of the aggregators because it uses it to correctly merge
* It uses Arrow's native dynamically typed object, `Array`.
* Arrow shines in batch operations and both `merge` and `concatenate` of uniform types are very performant.
Example: average
* the state is `n: u32` and `sum: f64`
* For every batch, we update them accordingly.
* At the end of the accumulation (of a partition), we convert `n` and `sum` to a RecordBatch of 1 row and two columns: `[n, sum]`
* The RecordBatch is (sent back / transmitted over network)
* Once all N record batches arrive, `merge` is performed, which builds a RecordBatch with N rows and 2 columns.
* Finally, `get_value` returns an array with one entry computed from the state
*/
pin_project! {
struct GroupedHashAggregateStream {
schema: SchemaRef,
#[pin]
output: futures::channel::oneshot::Receiver<ArrowResult<RecordBatch>>,
finished: bool,
output_rows: Arc<SQLMetric>,
}
}
pub(crate) fn group_aggregate_batch(
mode: &AggregateMode,
group_expr: &[Arc<dyn PhysicalExpr>],
batch: RecordBatch,
mut accumulation_state: AccumulationState,
aggregate_expressions: &[Vec<Arc<dyn PhysicalExpr>>],
skip_row: impl Fn(&RecordBatch, /*row_index*/ usize) -> bool,
) -> Result<AccumulationState> {
// Note: There is some parallel array &[Arc<dyn AggregateExpr>] that simply isn't passed to this
// function, but which exists and might be useful.
// evaluate the grouping expressions
let group_values = evaluate(group_expr, &batch)?;
// evaluate the aggregation expressions.
// We could evaluate them after the `take`, but since we need to evaluate all
// of them anyways, it is more performant to do it while they are together.
let aggr_input_values = evaluate_many(aggregate_expressions, &batch)?;
// create vector large enough to hold the grouping key
// this is an optimization to avoid allocating `key` on every row.
// it will be overwritten on every iteration of the loop below
let mut group_by_values = smallvec![GroupByScalar::UInt32(0); group_values.len()];
let mut key = SmallVec::new();
// 1.1 construct the key from the group values
// 1.2 construct the mapping key if it does not exist
// 1.3 add the row' index to `indices`
// Keys received in this batch
let mut batch_keys = BinaryBuilder::new(0);
for row in 0..batch.num_rows() {
if skip_row(&batch, row) {
continue;
}
// 1.1
create_key(&group_values, row, &mut key)
.map_err(DataFusionError::into_arrow_external_error)?;
accumulation_state
.accumulators
.raw_entry_mut()
.from_key(&key)
// 1.3
.and_modify(|_, AccumulationGroupState { indices: v, .. }| {
if v.is_empty() {
batch_keys.append_value(&key).expect("must not fail");
};
v.push(row as u32)
})
// 1.2
.or_insert_with(|| {
batch_keys.append_value(&key).expect("must not fail");
// Note that we still use plain String objects in GroupByScalar. Thus flattened_group_by_values isn't that great.
let _ = create_group_by_values(&group_values, row, &mut group_by_values);
accumulation_state.flattened_group_by_values.extend(
group_by_values
.iter_mut()
.map(|x| std::mem::replace(x, GroupByScalar::UInt32(0))),
);
let group_index = accumulation_state.next_group_index;
accumulation_state.next_group_index += 1;
(
key.clone(),
AccumulationGroupState {
indices: smallvec![row as u32],
group_index,
},
)
});
}
// Collect all indices + offsets based on keys in this vec
let mut batch_indices: UInt32Builder = UInt32Builder::new(0);
let mut all_group_indices = Vec::new();
let mut offsets = vec![0];
let mut offset_so_far = 0;
let batch_keys = batch_keys.finish();
for key in batch_keys.iter() {
let key = key.unwrap();
let AccumulationGroupState {
indices,
group_index,
..
} = accumulation_state.accumulators.get_mut(key).unwrap();
batch_indices.append_slice(indices)?;
all_group_indices.extend(std::iter::repeat(*group_index).take(indices.len()));
offset_so_far += indices.len();
offsets.push(offset_so_far);
indices.clear();
}
let batch_indices = batch_indices.finish();
// `Take` all values based on indices into Arrays
let values: Vec<Vec<Arc<dyn Array>>> = aggr_input_values
.iter()
.map(|array| {
array
.iter()
.map(|array| {
compute::take(
array.as_ref(),
&batch_indices,
None, // None: no index check
)
.unwrap()
})
.collect()
// 2.3
})
.collect();
for (accumulator_index, accumulator) in accumulation_state
.groups_accumulators
.iter_mut()
.enumerate()
{
match mode {
AggregateMode::Partial | AggregateMode::Full => accumulator
.update_batch_preordered(
&values[accumulator_index],
&all_group_indices,
&offsets,
None,
accumulation_state.next_group_index,
)?,
AggregateMode::FinalPartitioned | AggregateMode::Final => {
// note: the aggregation here is over states, not values, thus the merge
accumulator.merge_batch_preordered(
&values[accumulator_index],
&all_group_indices,
&offsets,
None,
accumulation_state.next_group_index,
)?
}
}
}
Ok(accumulation_state)
}
/// Appends a sequence of [u8] bytes for the value in `col[row]` to
/// `vec` to be used as a key into the hash map for a dictionary type
///
/// Note that ideally, for dictionary encoded columns, we would be
/// able to simply use the dictionary idicies themselves (no need to
/// look up values) or possibly simply build the hash table entirely
/// on the dictionary indexes.
///
/// This aproach would likely work (very) well for the common case,
/// but it also has to to handle the case where the dictionary itself
/// is not the same across all record batches (and thus indexes in one
/// record batch may not correspond to the same index in another)
fn dictionary_create_key_for_col<K: ArrowDictionaryKeyType>(
col: &ArrayRef,
row: usize,
vec: &mut KeyVec,
) -> Result<()> {
let dict_col = col.as_any().downcast_ref::<DictionaryArray<K>>().unwrap();
// look up the index in the values dictionary
let keys_col = dict_col.keys();
let values_index = keys_col.value(row).to_usize().ok_or_else(|| {
DataFusionError::Internal(format!(
"Can not convert index to usize in dictionary of type creating group by value {:?}",
keys_col.data_type()
))
})?;
create_key_for_col(dict_col.values(), values_index, vec)
}
/// Appends a sequence of [u8] bytes for the value in `col[row]` to
/// `vec` to be used as a key into the hash map
fn create_key_for_col(col: &ArrayRef, row: usize, vec: &mut KeyVec) -> Result<()> {
match col.data_type() {
DataType::Boolean => {
let array = col.as_any().downcast_ref::<BooleanArray>().unwrap();
vec.extend_from_slice(&[array.value(row) as u8]);
}
DataType::Float32 => {
let array = col.as_any().downcast_ref::<Float32Array>().unwrap();
vec.extend_from_slice(&array.value(row).to_le_bytes());
}
DataType::Float64 => {
let array = col.as_any().downcast_ref::<Float64Array>().unwrap();
vec.extend_from_slice(&array.value(row).to_le_bytes());
}
DataType::UInt8 => {
let array = col.as_any().downcast_ref::<UInt8Array>().unwrap();
vec.extend_from_slice(&array.value(row).to_le_bytes());
}
DataType::UInt16 => {
let array = col.as_any().downcast_ref::<UInt16Array>().unwrap();
vec.extend_from_slice(&array.value(row).to_le_bytes());
}
DataType::UInt32 => {
let array = col.as_any().downcast_ref::<UInt32Array>().unwrap();
vec.extend_from_slice(&array.value(row).to_le_bytes());
}
DataType::UInt64 => {
let array = col.as_any().downcast_ref::<UInt64Array>().unwrap();
vec.extend_from_slice(&array.value(row).to_le_bytes());
}
DataType::Int8 => {
let array = col.as_any().downcast_ref::<Int8Array>().unwrap();
vec.extend_from_slice(&array.value(row).to_le_bytes());
}
DataType::Int16 => {
let array = col.as_any().downcast_ref::<Int16Array>().unwrap();
vec.extend_from_slice(&array.value(row).to_le_bytes());
}
DataType::Int32 => {
let array = col.as_any().downcast_ref::<Int32Array>().unwrap();
vec.extend_from_slice(&array.value(row).to_le_bytes());
}
DataType::Int64 => {
let array = col.as_any().downcast_ref::<Int64Array>().unwrap();
vec.extend_from_slice(&array.value(row).to_le_bytes());
}
DataType::Int96 => {
let array = col.as_any().downcast_ref::<Int96Array>().unwrap();
vec.extend_from_slice(&array.value(row).to_le_bytes());
}
DataType::Timestamp(TimeUnit::Millisecond, None) => {
let array = col
.as_any()
.downcast_ref::<TimestampMillisecondArray>()
.unwrap();
vec.extend_from_slice(&array.value(row).to_le_bytes());
}
DataType::Timestamp(TimeUnit::Microsecond, None) => {
let array = col
.as_any()
.downcast_ref::<TimestampMicrosecondArray>()
.unwrap();
vec.extend_from_slice(&array.value(row).to_le_bytes());
}
DataType::Timestamp(TimeUnit::Nanosecond, None) => {
let array = col
.as_any()
.downcast_ref::<TimestampNanosecondArray>()
.unwrap();
vec.extend_from_slice(&array.value(row).to_le_bytes());
}
DataType::Utf8 => {
let array = col.as_any().downcast_ref::<StringArray>().unwrap();
let value = array.value(row);
// store the size
vec.extend_from_slice(&value.len().to_le_bytes());
// store the string value
vec.extend_from_slice(value.as_bytes());
}
DataType::LargeUtf8 => {
let array = col.as_any().downcast_ref::<LargeStringArray>().unwrap();
let value = array.value(row);
// store the size
vec.extend_from_slice(&value.len().to_le_bytes());
// store the string value
vec.extend_from_slice(value.as_bytes());
}
DataType::Date32 => {
let array = col.as_any().downcast_ref::<Date32Array>().unwrap();
vec.extend_from_slice(&array.value(row).to_le_bytes());
}
DataType::Int64Decimal(0) => {
let array = col.as_any().downcast_ref::<Int64Decimal0Array>().unwrap();
vec.extend_from_slice(&array.value(row).to_le_bytes());
}
DataType::Int64Decimal(1) => {
let array = col.as_any().downcast_ref::<Int64Decimal1Array>().unwrap();
vec.extend_from_slice(&array.value(row).to_le_bytes());
}
DataType::Int64Decimal(2) => {
let array = col.as_any().downcast_ref::<Int64Decimal2Array>().unwrap();
vec.extend_from_slice(&array.value(row).to_le_bytes());
}
DataType::Int64Decimal(3) => {
let array = col.as_any().downcast_ref::<Int64Decimal3Array>().unwrap();
vec.extend_from_slice(&array.value(row).to_le_bytes());
}
DataType::Int64Decimal(4) => {
let array = col.as_any().downcast_ref::<Int64Decimal4Array>().unwrap();
vec.extend_from_slice(&array.value(row).to_le_bytes());
}
DataType::Int64Decimal(5) => {
let array = col.as_any().downcast_ref::<Int64Decimal5Array>().unwrap();
vec.extend_from_slice(&array.value(row).to_le_bytes());
}
DataType::Int64Decimal(10) => {
let array = col.as_any().downcast_ref::<Int64Decimal10Array>().unwrap();
vec.extend_from_slice(&array.value(row).to_le_bytes());
}
DataType::Int96Decimal(0) => {
let array = col.as_any().downcast_ref::<Int96Decimal0Array>().unwrap();
vec.extend_from_slice(&array.value(row).to_le_bytes());
}
DataType::Int96Decimal(1) => {
let array = col.as_any().downcast_ref::<Int96Decimal1Array>().unwrap();
vec.extend_from_slice(&array.value(row).to_le_bytes());
}
DataType::Int96Decimal(2) => {
let array = col.as_any().downcast_ref::<Int96Decimal2Array>().unwrap();
vec.extend_from_slice(&array.value(row).to_le_bytes());
}
DataType::Int96Decimal(3) => {
let array = col.as_any().downcast_ref::<Int96Decimal3Array>().unwrap();
vec.extend_from_slice(&array.value(row).to_le_bytes());
}
DataType::Int96Decimal(4) => {
let array = col.as_any().downcast_ref::<Int96Decimal4Array>().unwrap();
vec.extend_from_slice(&array.value(row).to_le_bytes());
}
DataType::Int96Decimal(5) => {
let array = col.as_any().downcast_ref::<Int96Decimal5Array>().unwrap();
vec.extend_from_slice(&array.value(row).to_le_bytes());
}
DataType::Int96Decimal(10) => {
let array = col.as_any().downcast_ref::<Int96Decimal10Array>().unwrap();
vec.extend_from_slice(&array.value(row).to_le_bytes());
}
DataType::Dictionary(index_type, _) => match **index_type {
DataType::Int8 => {
dictionary_create_key_for_col::<Int8Type>(col, row, vec)?;
}
DataType::Int16 => {
dictionary_create_key_for_col::<Int16Type>(col, row, vec)?;
}
DataType::Int32 => {
dictionary_create_key_for_col::<Int32Type>(col, row, vec)?;
}
DataType::Int64 => {
dictionary_create_key_for_col::<Int64Type>(col, row, vec)?;
}
DataType::UInt8 => {
dictionary_create_key_for_col::<UInt8Type>(col, row, vec)?;
}
DataType::UInt16 => {
dictionary_create_key_for_col::<UInt16Type>(col, row, vec)?;
}
DataType::UInt32 => {
dictionary_create_key_for_col::<UInt32Type>(col, row, vec)?;
}
DataType::UInt64 => {
dictionary_create_key_for_col::<UInt64Type>(col, row, vec)?;
}
_ => {
return Err(DataFusionError::Internal(format!(
"Unsupported GROUP BY type (dictionary index type not supported creating key) {}",
col.data_type(),
)))
}
},
_ => {
// This is internal because we should have caught this before.
return Err(DataFusionError::Internal(format!(
"Unsupported GROUP BY type creating key {}",
col.data_type(),
)));
}
}
Ok(())
}
/// Create a key `Vec<u8>` that is used as key for the hashmap
pub(crate) fn create_key(
group_by_keys: &[ArrayRef],
row: usize,
vec: &mut KeyVec,
) -> Result<()> {
vec.clear();
for col in group_by_keys {
create_key_for_col(col, row, vec)?
}
Ok(())
}
#[tracing::instrument(level = "trace", skip(schema, group_expr, aggr_expr, input))]
async fn compute_grouped_hash_aggregate(
mode: AggregateMode,
schema: SchemaRef,
group_expr: Vec<Arc<dyn PhysicalExpr>>,
aggr_expr: Vec<Arc<dyn AggregateExpr>>,
mut input: SendableRecordBatchStream,
) -> ArrowResult<RecordBatch> {
// The expressions to evaluate the batch, one vec of expressions per aggregation.
// Assume create_schema() always put group columns in front of aggr columns, we set
// col_idx_base to group expression count.
let aggregate_expressions =
aggregate_expressions(&aggr_expr, &mode, group_expr.len())
.map_err(DataFusionError::into_arrow_external_error)?;
// mapping key -> (set of accumulators, indices of the key in the batch)
// * the indexes are updated at each row
// * the accumulators are updated at the end of each batch
// * the indexes are `clear`ed at the end of each batch
//let mut accumulators: Accumulators = FnvHashMap::default();
// iterate over all input batches and update the accumulators
let mut accumulators = create_accumulation_state(&aggr_expr)?;
while let Some(batch) = input.next().await {
let batch = batch?;
accumulators = group_aggregate_batch(
&mode,
&group_expr,
batch,
accumulators,
&aggregate_expressions,
|_, _| false,
)
.map_err(DataFusionError::into_arrow_external_error)?;
}
create_batch_from_map(&mode, accumulators, group_expr.len(), &schema)
}
impl GroupedHashAggregateStream {
/// Create a new HashAggregateStream
pub fn new(
strategy: AggregateStrategy,
mode: AggregateMode,
schema: SchemaRef,
group_expr: Vec<Arc<dyn PhysicalExpr>>,
aggr_expr: Vec<Arc<dyn AggregateExpr>>,
input: SendableRecordBatchStream,
output_rows: Arc<SQLMetric>,
) -> Self {
let (tx, rx) = futures::channel::oneshot::channel();
let schema_clone = schema.clone();
let task = async move {
match strategy {
AggregateStrategy::Hash => {
compute_grouped_hash_aggregate(
mode,
schema_clone,
group_expr,
aggr_expr,
input,
)
.await
}
AggregateStrategy::InplaceSorted => {
compute_grouped_sorted_aggregate(
mode,
schema_clone,
group_expr,
aggr_expr,
input,
)
.await
}
}
};
cube_ext::spawn_oneshot_with_catch_unwind(task, tx);
Self {
schema,
output: rx,
finished: false,
output_rows,
}
}
}
#[allow(missing_docs)]
pub type KeyVec = SmallVec<[u8; 64]>;
type AccumulatorItem = Box<dyn Accumulator>;
#[allow(missing_docs)]
pub type AccumulatorSet = SmallVec<[AccumulatorItem; 2]>;
#[allow(missing_docs)]
pub type Accumulators = HashMap<KeyVec, AccumulationGroupState, RandomState>;
#[allow(missing_docs)]
pub struct AccumulationGroupState {
indices: SmallVec<[u32; 4]>,
group_index: usize,
}
#[allow(missing_docs)]
#[derive(Default)]
pub struct AccumulationState {
accumulators: HashMap<KeyVec, AccumulationGroupState, RandomState>,
// Of length accumulators.len() * N where N is the number of group by columns.
flattened_group_by_values: Vec<GroupByScalar>,
groups_accumulators: Vec<Box<dyn GroupsAccumulator>>,
// For now, always equal to accumulators.len()
next_group_index: usize,
}
impl AccumulationState {
/// Constructs an initial AccumulationState.
pub fn new(
groups_accumulators: Vec<Box<dyn GroupsAccumulator>>,
) -> AccumulationState {
AccumulationState {
accumulators: HashMap::new(),
flattened_group_by_values: Vec::new(),
groups_accumulators,
next_group_index: 0,
}
}
}
impl Stream for GroupedHashAggregateStream {
type Item = ArrowResult<RecordBatch>;
fn poll_next(
self: std::pin::Pin<&mut Self>,
cx: &mut Context<'_>,
) -> Poll<Option<Self::Item>> {
if self.finished {
return Poll::Ready(None);
}
let output_rows = self.output_rows.clone();
// is the output ready?
let this = self.project();
let output_poll = this.output.poll(cx);
match output_poll {
Poll::Ready(result) => {
*this.finished = true;
// check for error in receiving channel and unwrap actual result
let result = match result {
Err(e) => Err(ArrowError::ExternalError(Box::new(e))), // error receiving
Ok(result) => result,
};
if let Ok(batch) = &result {
output_rows.add(batch.num_rows())
}
Poll::Ready(Some(result))
}
Poll::Pending => Poll::Pending,
}
}
}
impl RecordBatchStream for GroupedHashAggregateStream {
fn schema(&self) -> SchemaRef {
self.schema.clone()
}
}
/// Evaluates expressions against a record batch.
pub(crate) fn evaluate(
expr: &[Arc<dyn PhysicalExpr>],
batch: &RecordBatch,
) -> Result<Vec<ArrayRef>> {
expr.iter()
.map(|expr| expr.evaluate(batch))
.map(|r| r.map(|v| v.into_array(batch.num_rows())))
.collect::<Result<Vec<_>>>()
}
/// Evaluates expressions against a record batch.
pub(crate) fn evaluate_many(
expr: &[Vec<Arc<dyn PhysicalExpr>>],
batch: &RecordBatch,
) -> Result<Vec<Vec<ArrayRef>>> {
expr.iter()
.map(|expr| evaluate(expr, batch))
.collect::<Result<Vec<_>>>()
}
/// uses `state_fields` to build a vec of physical column expressions required to merge the
/// AggregateExpr' accumulator's state.
///
/// `index_base` is the starting physical column index for the next expanded state field.
fn merge_expressions(
index_base: usize,
expr: &Arc<dyn AggregateExpr>,
) -> Result<Vec<Arc<dyn PhysicalExpr>>> {
Ok(expr
.state_fields()?
.iter()
.enumerate()
.map(|(idx, f)| {
Arc::new(Column::new(f.name(), index_base + idx)) as Arc<dyn PhysicalExpr>
})
.collect::<Vec<_>>())
}