Deprecated: The each() function is deprecated. This message will be suppressed on further calls in /home/zhenxiangba/zhenxiangba.com/public_html/phproxy-improved-master/index.php on line 456
datafusion 0.15.0 - Docs.rs
[go: Go Back, main page]

datafusion 0.15.0

DataFusion is an in-memory query engine that uses Apache Arrow as the memory model
Documentation
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
// 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::cell::RefCell;
use std::rc::Rc;
use std::str;
use std::sync::{Arc, Mutex};

use crate::error::{ExecutionError, Result};
use crate::execution::physical_plan::{
    Accumulator, AggregateExpr, BatchIterator, ExecutionPlan, Partition, PhysicalExpr,
};

use arrow::array::{
    ArrayRef, BinaryArray, Int16Array, Int32Array, Int64Array, Int8Array, UInt16Array,
    UInt32Array, UInt64Array, UInt8Array,
};
use arrow::array::{
    BinaryBuilder, Float32Builder, Float64Builder, Int16Builder, Int32Builder,
    Int64Builder, Int8Builder, UInt16Builder, UInt32Builder, UInt64Builder, UInt8Builder,
};
use arrow::datatypes::{DataType, Field, Schema};
use arrow::record_batch::RecordBatch;

use crate::execution::physical_plan::expressions::Column;
use crate::execution::physical_plan::merge::MergeExec;
use crate::logicalplan::ScalarValue;
use fnv::FnvHashMap;

/// Hash aggregate execution plan
pub struct HashAggregateExec {
    group_expr: Vec<Arc<dyn PhysicalExpr>>,
    aggr_expr: Vec<Arc<dyn AggregateExpr>>,
    input: Arc<dyn ExecutionPlan>,
    schema: Arc<Schema>,
}

impl HashAggregateExec {
    /// Create a new hash aggregate execution plan
    pub fn try_new(
        group_expr: Vec<Arc<dyn PhysicalExpr>>,
        aggr_expr: Vec<Arc<dyn AggregateExpr>>,
        input: Arc<dyn ExecutionPlan>,
        schema: Arc<Schema>,
    ) -> Result<Self> {
        Ok(HashAggregateExec {
            group_expr,
            aggr_expr,
            input,
            schema,
        })
    }
}

impl ExecutionPlan for HashAggregateExec {
    fn schema(&self) -> Arc<Schema> {
        self.schema.clone()
    }

    fn partitions(&self) -> Result<Vec<Arc<dyn Partition>>> {
        let partitions: Vec<Arc<dyn Partition>> = self
            .input
            .partitions()?
            .iter()
            .map(|p| {
                let aggregate: Arc<dyn Partition> =
                    Arc::new(HashAggregatePartition::new(
                        self.group_expr.clone(),
                        self.aggr_expr.clone(),
                        p.clone() as Arc<dyn Partition>,
                        self.schema.clone(),
                    ));

                aggregate
            })
            .collect();

        // create partition to combine and aggregate the results
        let final_group: Vec<Arc<dyn PhysicalExpr>> = (0..self.group_expr.len())
            .map(|i| Arc::new(Column::new(i)) as Arc<dyn PhysicalExpr>)
            .collect();

        let final_aggr: Vec<Arc<dyn AggregateExpr>> = (0..self.aggr_expr.len())
            .map(|i| {
                let aggr = self.aggr_expr[i].create_combiner(i + self.group_expr.len());
                aggr as Arc<dyn AggregateExpr>
            })
            .collect();

        let mut fields = vec![];
        for expr in &final_group {
            let name = expr.name();
            fields.push(Field::new(&name, expr.data_type(&self.schema)?, true));
        }
        for expr in &final_aggr {
            let name = expr.name();
            fields.push(Field::new(&name, expr.data_type(&self.schema)?, true));
        }
        let schema = Arc::new(Schema::new(fields));

        let merge = MergeExec::new(schema.clone(), partitions);
        let merged: Vec<Arc<dyn Partition>> = merge.partitions()?;
        if merged.len() == 1 {
            Ok(vec![Arc::new(HashAggregatePartition::new(
                final_group,
                final_aggr,
                merged[0].clone(),
                schema,
            ))])
        } else {
            Err(ExecutionError::InternalError(format!(
                "MergeExec returned {} partitions",
                merged.len()
            )))
        }
    }
}

struct HashAggregatePartition {
    group_expr: Vec<Arc<dyn PhysicalExpr>>,
    aggr_expr: Vec<Arc<dyn AggregateExpr>>,
    input: Arc<dyn Partition>,
    schema: Arc<Schema>,
}

impl HashAggregatePartition {
    /// Create a new HashAggregatePartition
    pub fn new(
        group_expr: Vec<Arc<dyn PhysicalExpr>>,
        aggr_expr: Vec<Arc<dyn AggregateExpr>>,
        input: Arc<dyn Partition>,
        schema: Arc<Schema>,
    ) -> Self {
        HashAggregatePartition {
            group_expr,
            aggr_expr,
            input,
            schema,
        }
    }
}

impl Partition for HashAggregatePartition {
    fn execute(&self) -> Result<Arc<Mutex<dyn BatchIterator>>> {
        if self.group_expr.is_empty() {
            Ok(Arc::new(Mutex::new(HashAggregateIterator::new(
                self.schema.clone(),
                self.aggr_expr.clone(),
                self.input.execute()?,
            ))))
        } else {
            Ok(Arc::new(Mutex::new(GroupedHashAggregateIterator::new(
                self.schema.clone(),
                self.group_expr.clone(),
                self.aggr_expr.clone(),
                self.input.execute()?,
            ))))
        }
    }
}

/// Create array from `key` attribute in map entry (representing a grouping scalar value)
macro_rules! group_array_from_map_entries {
    ($BUILDER:ident, $TY:ident, $ENTRIES:expr, $COL_INDEX:expr) => {{
        let mut builder = $BUILDER::new($ENTRIES.len());
        let mut err = false;
        for j in 0..$ENTRIES.len() {
            match $ENTRIES[j].k[$COL_INDEX] {
                GroupByScalar::$TY(n) => builder.append_value(n).unwrap(),
                _ => err = true,
            }
        }
        if err {
            Err(ExecutionError::ExecutionError(
                "unexpected type when creating grouping array from aggregate map"
                    .to_string(),
            ))
        } else {
            Ok(Arc::new(builder.finish()) as ArrayRef)
        }
    }};
}

/// Create array from `value` attribute in map entry (representing an aggregate scalar
/// value)
macro_rules! aggr_array_from_map_entries {
    ($BUILDER:ident, $TY:ident, $TY2:ty, $ENTRIES:expr, $COL_INDEX:expr) => {{
        let mut builder = $BUILDER::new($ENTRIES.len());
        let mut err = false;
        for j in 0..$ENTRIES.len() {
            match $ENTRIES[j].v[$COL_INDEX] {
                Some(ScalarValue::$TY(n)) => builder.append_value(n as $TY2).unwrap(),
                None => builder.append_null().unwrap(),
                _ => err = true,
            }
        }
        if err {
            Err(ExecutionError::ExecutionError(
                "unexpected type when creating aggregate array from aggregate map"
                    .to_string(),
            ))
        } else {
            Ok(Arc::new(builder.finish()) as ArrayRef)
        }
    }};
}

#[derive(Debug)]
struct MapEntry {
    k: Vec<GroupByScalar>,
    v: Vec<Option<ScalarValue>>,
}

struct GroupedHashAggregateIterator {
    schema: Arc<Schema>,
    group_expr: Vec<Arc<dyn PhysicalExpr>>,
    aggr_expr: Vec<Arc<dyn AggregateExpr>>,
    input: Arc<Mutex<dyn BatchIterator>>,
    finished: bool,
}

impl GroupedHashAggregateIterator {
    /// Create a new HashAggregateIterator
    pub fn new(
        schema: Arc<Schema>,
        group_expr: Vec<Arc<dyn PhysicalExpr>>,
        aggr_expr: Vec<Arc<dyn AggregateExpr>>,
        input: Arc<Mutex<dyn BatchIterator>>,
    ) -> Self {
        GroupedHashAggregateIterator {
            schema,
            group_expr,
            aggr_expr,
            input,
            finished: false,
        }
    }
}

impl BatchIterator for GroupedHashAggregateIterator {
    fn schema(&self) -> Arc<Schema> {
        self.schema.clone()
    }

    fn next(&mut self) -> Result<Option<RecordBatch>> {
        if self.finished {
            return Ok(None);
        }

        self.finished = true;

        // create map to store accumulators for each unique grouping key
        let mut map: FnvHashMap<Vec<GroupByScalar>, Vec<Rc<RefCell<dyn Accumulator>>>> =
            FnvHashMap::default();

        // iterate over all input batches and update the accumulators
        let mut input = self.input.lock().unwrap();

        // iterate over input and perform aggregation
        while let Some(batch) = input.next()? {
            // evaluate the grouping expressions for this batch
            let group_values = self
                .group_expr
                .iter()
                .map(|expr| expr.evaluate(&batch))
                .collect::<Result<Vec<_>>>()?;

            // iterate over each row in the batch
            for row in 0..batch.num_rows() {
                // create grouping key for this row
                let key = create_key(&group_values, row)?;

                let updated: Result<bool> = match map.get(&key) {
                    Some(accumulators) => {
                        let _ = accumulators
                            .iter()
                            .map(|accum| accum.borrow_mut().accumulate(&batch, row))
                            .collect::<Result<Vec<_>>>()?;
                        Ok(true)
                    }
                    None => Ok(false),
                };

                if !updated? {
                    let accumulators: Vec<Rc<RefCell<dyn Accumulator>>> = self
                        .aggr_expr
                        .iter()
                        .map(|expr| expr.create_accumulator())
                        .collect();

                    let _ = accumulators
                        .iter()
                        .map(|accum| accum.borrow_mut().accumulate(&batch, row))
                        .collect::<Result<Vec<_>>>()?;

                    map.insert(key.clone(), accumulators);
                }
            }
        }
        let input_schema = input.schema();

        // convert the map to a vec to make it easier to build arrays
        let entries: Vec<MapEntry> = map
            .iter()
            .map(|(k, v)| {
                let aggr_values = v
                    .iter()
                    .map(|accum| {
                        let accum = accum.borrow_mut();
                        accum.get_value()
                    })
                    .collect::<Result<Vec<_>>>()?;

                Ok(MapEntry {
                    k: k.clone(),
                    v: aggr_values,
                })
            })
            .collect::<Result<Vec<_>>>()?;

        // build the result arrays
        let mut result_arrays: Vec<ArrayRef> =
            Vec::with_capacity(self.group_expr.len() + self.aggr_expr.len());

        // grouping values
        for i in 0..self.group_expr.len() {
            let array: Result<ArrayRef> = match self.group_expr[i]
                .data_type(&input_schema)?
            {
                DataType::UInt8 => {
                    group_array_from_map_entries!(UInt8Builder, UInt8, entries, i)
                }
                DataType::UInt16 => {
                    group_array_from_map_entries!(UInt16Builder, UInt16, entries, i)
                }
                DataType::UInt32 => {
                    group_array_from_map_entries!(UInt32Builder, UInt32, entries, i)
                }
                DataType::UInt64 => {
                    group_array_from_map_entries!(UInt64Builder, UInt64, entries, i)
                }
                DataType::Int8 => {
                    group_array_from_map_entries!(Int8Builder, Int8, entries, i)
                }
                DataType::Int16 => {
                    group_array_from_map_entries!(Int16Builder, Int16, entries, i)
                }
                DataType::Int32 => {
                    group_array_from_map_entries!(Int32Builder, Int32, entries, i)
                }
                DataType::Int64 => {
                    group_array_from_map_entries!(Int64Builder, Int64, entries, i)
                }
                DataType::Utf8 => {
                    let mut builder = BinaryBuilder::new(1);
                    for j in 0..entries.len() {
                        match &entries[j].k[i] {
                            GroupByScalar::Utf8(s) => builder.append_string(&s).unwrap(),
                            _ => {}
                        }
                    }
                    Ok(Arc::new(builder.finish()) as ArrayRef)
                }
                _ => Err(ExecutionError::ExecutionError(
                    "Unsupported group by expr".to_string(),
                )),
            };
            result_arrays.push(array?);

            // aggregate values
            for i in 0..self.aggr_expr.len() {
                let aggr_data_type = self.aggr_expr[i].data_type(&input_schema)?;
                let array = match aggr_data_type {
                    DataType::UInt8 => aggr_array_from_map_entries!(
                        UInt64Builder,
                        UInt8,
                        u64,
                        entries,
                        i
                    ),
                    DataType::UInt16 => aggr_array_from_map_entries!(
                        UInt64Builder,
                        UInt16,
                        u64,
                        entries,
                        i
                    ),
                    DataType::UInt32 => aggr_array_from_map_entries!(
                        UInt64Builder,
                        UInt32,
                        u64,
                        entries,
                        i
                    ),
                    DataType::UInt64 => aggr_array_from_map_entries!(
                        UInt64Builder,
                        UInt64,
                        u64,
                        entries,
                        i
                    ),
                    DataType::Int8 => {
                        aggr_array_from_map_entries!(Int64Builder, Int8, i64, entries, i)
                    }
                    DataType::Int16 => {
                        aggr_array_from_map_entries!(Int64Builder, Int16, i64, entries, i)
                    }
                    DataType::Int32 => {
                        aggr_array_from_map_entries!(Int64Builder, Int32, i64, entries, i)
                    }
                    DataType::Int64 => {
                        aggr_array_from_map_entries!(Int64Builder, Int64, i64, entries, i)
                    }
                    DataType::Float32 => aggr_array_from_map_entries!(
                        Float32Builder,
                        Float32,
                        f32,
                        entries,
                        i
                    ),
                    DataType::Float64 => aggr_array_from_map_entries!(
                        Float64Builder,
                        Float64,
                        f64,
                        entries,
                        i
                    ),
                    _ => Err(ExecutionError::ExecutionError(
                        "Unsupported aggregate expr".to_string(),
                    )),
                };
                result_arrays.push(array?);
            }
        }

        let mut fields = vec![];
        for expr in &self.group_expr {
            let name = expr.name();
            fields.push(Field::new(&name, expr.data_type(&input_schema)?, true))
        }
        for expr in &self.aggr_expr {
            let name = expr.name();
            fields.push(Field::new(&name, expr.data_type(&input_schema)?, true))
        }
        let schema = Schema::new(fields);

        let batch = RecordBatch::try_new(Arc::new(schema), result_arrays)?;
        Ok(Some(batch))
    }
}

struct HashAggregateIterator {
    schema: Arc<Schema>,
    aggr_expr: Vec<Arc<dyn AggregateExpr>>,
    input: Arc<Mutex<dyn BatchIterator>>,
    finished: bool,
}

impl HashAggregateIterator {
    /// Create a new HashAggregateIterator
    pub fn new(
        schema: Arc<Schema>,
        aggr_expr: Vec<Arc<dyn AggregateExpr>>,
        input: Arc<Mutex<dyn BatchIterator>>,
    ) -> Self {
        HashAggregateIterator {
            schema,
            aggr_expr,
            input,
            finished: false,
        }
    }
}

impl BatchIterator for HashAggregateIterator {
    fn schema(&self) -> Arc<Schema> {
        self.schema.clone()
    }

    fn next(&mut self) -> Result<Option<RecordBatch>> {
        if self.finished {
            return Ok(None);
        }

        self.finished = true;

        let accumulators: Vec<Rc<RefCell<dyn Accumulator>>> = self
            .aggr_expr
            .iter()
            .map(|expr| expr.create_accumulator())
            .collect();

        // iterate over all input batches and update the accumulators
        let mut input = self.input.lock().unwrap();

        // iterate over input and perform aggregation
        while let Some(batch) = input.next()? {
            // iterate over each row in the batch
            for row in 0..batch.num_rows() {
                let _ = accumulators
                    .iter()
                    .map(|accum| accum.borrow_mut().accumulate(&batch, row))
                    .collect::<Result<Vec<_>>>()?;
            }
        }

        let input_schema = input.schema();

        // build the result arrays
        let mut result_arrays: Vec<ArrayRef> = Vec::with_capacity(self.aggr_expr.len());

        let entries = vec![MapEntry {
            k: vec![],
            v: accumulators
                .iter()
                .map(|accum| accum.borrow_mut().get_value())
                .collect::<Result<Vec<_>>>()?,
        }];

        // aggregate values
        for i in 0..self.aggr_expr.len() {
            let aggr_data_type = self.aggr_expr[i].data_type(&input_schema)?;
            let array = match aggr_data_type {
                DataType::UInt8 => {
                    aggr_array_from_map_entries!(UInt64Builder, UInt8, u64, entries, i)
                }
                DataType::UInt16 => {
                    aggr_array_from_map_entries!(UInt64Builder, UInt16, u64, entries, i)
                }
                DataType::UInt32 => {
                    aggr_array_from_map_entries!(UInt64Builder, UInt32, u64, entries, i)
                }
                DataType::UInt64 => {
                    aggr_array_from_map_entries!(UInt64Builder, UInt64, u64, entries, i)
                }
                DataType::Int8 => {
                    aggr_array_from_map_entries!(Int64Builder, Int8, i64, entries, i)
                }
                DataType::Int16 => {
                    aggr_array_from_map_entries!(Int64Builder, Int16, i64, entries, i)
                }
                DataType::Int32 => {
                    aggr_array_from_map_entries!(Int64Builder, Int32, i64, entries, i)
                }
                DataType::Int64 => {
                    aggr_array_from_map_entries!(Int64Builder, Int64, i64, entries, i)
                }
                DataType::Float32 => {
                    aggr_array_from_map_entries!(Float32Builder, Float32, f32, entries, i)
                }
                DataType::Float64 => {
                    aggr_array_from_map_entries!(Float64Builder, Float64, f64, entries, i)
                }
                _ => Err(ExecutionError::ExecutionError(
                    "Unsupported aggregate expr".to_string(),
                )),
            };
            result_arrays.push(array?);
        }

        let mut fields = vec![];
        for expr in &self.aggr_expr {
            let name = expr.name();
            fields.push(Field::new(&name, expr.data_type(&input_schema)?, true))
        }
        let schema = Schema::new(fields);

        let batch = RecordBatch::try_new(Arc::new(schema), result_arrays)?;
        Ok(Some(batch))
    }
}

/// Enumeration of types that can be used in a GROUP BY expression (all primitives except
/// for floating point numerics)
#[derive(Debug, PartialEq, Eq, Hash, Clone)]
enum GroupByScalar {
    UInt8(u8),
    UInt16(u16),
    UInt32(u32),
    UInt64(u64),
    Int8(i8),
    Int16(i16),
    Int32(i32),
    Int64(i64),
    Utf8(String),
}

/// Create a Vec<GroupByScalar> that can be used as a map key
fn create_key(group_by_keys: &Vec<ArrayRef>, row: usize) -> Result<Vec<GroupByScalar>> {
    group_by_keys
        .iter()
        .map(|col| match col.data_type() {
            DataType::UInt8 => {
                let array = col.as_any().downcast_ref::<UInt8Array>().unwrap();
                Ok(GroupByScalar::UInt8(array.value(row)))
            }
            DataType::UInt16 => {
                let array = col.as_any().downcast_ref::<UInt16Array>().unwrap();
                Ok(GroupByScalar::UInt16(array.value(row)))
            }
            DataType::UInt32 => {
                let array = col.as_any().downcast_ref::<UInt32Array>().unwrap();
                Ok(GroupByScalar::UInt32(array.value(row)))
            }
            DataType::UInt64 => {
                let array = col.as_any().downcast_ref::<UInt64Array>().unwrap();
                Ok(GroupByScalar::UInt64(array.value(row)))
            }
            DataType::Int8 => {
                let array = col.as_any().downcast_ref::<Int8Array>().unwrap();
                Ok(GroupByScalar::Int8(array.value(row)))
            }
            DataType::Int16 => {
                let array = col.as_any().downcast_ref::<Int16Array>().unwrap();
                Ok(GroupByScalar::Int16(array.value(row)))
            }
            DataType::Int32 => {
                let array = col.as_any().downcast_ref::<Int32Array>().unwrap();
                Ok(GroupByScalar::Int32(array.value(row)))
            }
            DataType::Int64 => {
                let array = col.as_any().downcast_ref::<Int64Array>().unwrap();
                Ok(GroupByScalar::Int64(array.value(row)))
            }
            DataType::Utf8 => {
                let array = col.as_any().downcast_ref::<BinaryArray>().unwrap();
                Ok(GroupByScalar::Utf8(String::from(
                    str::from_utf8(array.value(row)).unwrap(),
                )))
            }
            _ => Err(ExecutionError::ExecutionError(
                "Unsupported GROUP BY data type".to_string(),
            )),
        })
        .collect::<Result<Vec<GroupByScalar>>>()
}

#[cfg(test)]
mod tests {

    use super::*;
    use crate::execution::physical_plan::csv::CsvExec;
    use crate::execution::physical_plan::expressions::{col, sum};
    use crate::test;
    use arrow::datatypes::Field;

    #[test]
    fn aggregate() -> Result<()> {
        let schema = test::aggr_test_schema();

        let partitions = 4;
        let path = test::create_partitioned_csv("aggregate_test_100.csv", partitions)?;

        let csv = CsvExec::try_new(&path, schema, true, None, 1024)?;

        let group_expr: Vec<Arc<dyn PhysicalExpr>> = vec![col(1)];

        let aggr_expr: Vec<Arc<dyn AggregateExpr>> = vec![sum(col(3))];

        let schema = Arc::new(Schema::new(vec![
            Field::new("a", DataType::UInt32, true),
            Field::new("b", DataType::Int64, true),
        ]));

        let partition_aggregate = HashAggregateExec::try_new(
            group_expr.clone(),
            aggr_expr.clone(),
            Arc::new(csv),
            schema.clone(),
        )?;

        let result = test::execute(&partition_aggregate)?;
        assert_eq!(result.len(), 1);

        let batch = &result[0];
        assert_eq!(batch.num_columns(), 2);
        assert_eq!(batch.num_rows(), 5);

        let a = batch
            .column(0)
            .as_any()
            .downcast_ref::<UInt32Array>()
            .unwrap();
        let b = batch
            .column(1)
            .as_any()
            .downcast_ref::<Int64Array>()
            .unwrap();

        let mut group_values = vec![];
        for i in 0..a.len() {
            group_values.push(a.value(i))
        }

        let mut aggr_values = vec![];
        for i in 1..=5 {
            // find index of row with this value for the grouping column
            let index = group_values.iter().position(|&r| r == i).unwrap();
            aggr_values.push(b.value(index));
        }

        let expected: Vec<i64> = vec![88722, 90999, 80899, -120910, 92287];
        assert_eq!(aggr_values, expected);

        Ok(())
    }
}