forked from apache/datafusion
-
Notifications
You must be signed in to change notification settings - Fork 5
/
Copy pathmod.rs
720 lines (649 loc) · 24.9 KB
/
mod.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
// 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.
//! Traits for physical query plan, supporting parallel execution for partitioned relations.
use self::{
coalesce_partitions::CoalescePartitionsExec, display::DisplayableExecutionPlan,
};
use crate::physical_plan::expressions::PhysicalSortExpr;
use crate::{
error::{DataFusionError, Result},
scalar::ScalarValue,
};
use arrow::compute::kernels::partition::lexicographical_partition_ranges;
use arrow::compute::kernels::sort::{SortColumn, SortOptions};
use arrow::datatypes::{DataType, Schema, SchemaRef};
use arrow::error::Result as ArrowResult;
use arrow::record_batch::RecordBatch;
use arrow::{array::ArrayRef, datatypes::Field};
use async_trait::async_trait;
pub use display::DisplayFormatType;
use futures::stream::Stream;
use groups_accumulator::GroupsAccumulator;
use hashbrown::HashMap;
use std::fmt;
use std::fmt::{Debug, Display};
use std::ops::Range;
use std::sync::atomic::{AtomicUsize, Ordering};
use std::sync::Arc;
use std::{any::Any, pin::Pin};
/// Trait for types that stream [arrow::record_batch::RecordBatch]
pub trait RecordBatchStream: Stream<Item = ArrowResult<RecordBatch>> {
/// Returns the schema of this `RecordBatchStream`.
///
/// Implementation of this trait should guarantee that all `RecordBatch`'s returned by this
/// stream should have the same schema as returned from this method.
fn schema(&self) -> SchemaRef;
}
/// Trait for a stream of record batches.
pub type SendableRecordBatchStream = Pin<Box<dyn RecordBatchStream + Send>>;
/// SQL metric type
#[derive(Debug, Clone)]
pub enum MetricType {
/// Simple counter
Counter,
/// Wall clock time in nanoseconds
TimeNanos,
}
/// SQL metric such as counter (number of input or output rows) or timing information about
/// a physical operator.
#[derive(Debug)]
pub struct SQLMetric {
/// Metric value
value: AtomicUsize,
/// Metric type
metric_type: MetricType,
}
impl Clone for SQLMetric {
fn clone(&self) -> Self {
Self {
value: AtomicUsize::new(self.value.load(Ordering::Relaxed)),
metric_type: self.metric_type.clone(),
}
}
}
impl SQLMetric {
// relaxed ordering for operations on `value` poses no issues
// we're purely using atomic ops with no associated memory ops
/// Create a new metric for tracking a counter
pub fn counter() -> Arc<SQLMetric> {
Arc::new(SQLMetric::new(MetricType::Counter))
}
/// Create a new metric for tracking time in nanoseconds
pub fn time_nanos() -> Arc<SQLMetric> {
Arc::new(SQLMetric::new(MetricType::TimeNanos))
}
/// Create a new SQLMetric
pub fn new(metric_type: MetricType) -> Self {
Self {
value: AtomicUsize::new(0),
metric_type,
}
}
/// Add to the value
pub fn add(&self, n: usize) {
self.value.fetch_add(n, Ordering::Relaxed);
}
/// Add elapsed nanoseconds since `start`to self
pub fn add_elapsed(&self, start: std::time::Instant) {
self.add(start.elapsed().as_nanos() as usize)
}
/// Get the current value
pub fn value(&self) -> usize {
self.value.load(Ordering::Relaxed)
}
}
/// Physical planner interface
pub use self::planner::PhysicalPlanner;
use smallvec::SmallVec;
/// Various hints for planning and optimizations.
#[derive(Debug, Default, Clone, PartialEq, Eq)]
pub struct OptimizerHints {
/// If the output is sorted, contains indices of the sort key columns in the output schema.
/// Each partition should meet this sort order, but order between partitions is unspecified.
/// Note that this does **not** guarantee the exact ordering inside each of the columns, e.g.
/// the values may end up in ascending or descending order, nulls can go first or last.
pub sort_order: Option<Vec<usize>>,
/// Indices of columns that will always have the same value in each row. No information about
/// the value is provided.
pub single_value_columns: Vec<usize>,
}
/// `ExecutionPlan` represent nodes in the DataFusion Physical Plan.
///
/// Each `ExecutionPlan` is Partition-aware and is responsible for
/// creating the actual `async` [`SendableRecordBatchStream`]s
/// of [`RecordBatch`] that incrementally compute the operator's
/// output from its input partition.
///
/// [`ExecutionPlan`] can be displayed in an simplified form using the
/// return value from [`displayable`] in addition to the (normally
/// quite verbose) `Debug` output.
#[async_trait]
pub trait ExecutionPlan: Debug + Send + Sync {
/// Returns the execution plan as [`Any`](std::any::Any) so that it can be
/// downcast to a specific implementation.
fn as_any(&self) -> &dyn Any;
/// Get the schema for this execution plan
fn schema(&self) -> SchemaRef;
/// Specifies the output partitioning scheme of this plan
fn output_partitioning(&self) -> Partitioning;
/// Specifies the data distribution requirements of all the children for this operator
fn required_child_distribution(&self) -> Distribution {
Distribution::UnspecifiedDistribution
}
/// Get a list of child execution plans that provide the input for this plan. The returned list
/// will be empty for leaf nodes, will contain a single value for unary nodes, or two
/// values for binary nodes (such as joins).
fn children(&self) -> Vec<Arc<dyn ExecutionPlan>>;
/// Returns a new plan where all children were replaced by new plans.
/// The size of `children` must be equal to the size of `ExecutionPlan::children()`.
fn with_new_children(
&self,
children: Vec<Arc<dyn ExecutionPlan>>,
) -> Result<Arc<dyn ExecutionPlan>>;
/// Provides hints to the planner and the optimizer.
fn output_hints(&self) -> OptimizerHints {
OptimizerHints::default()
}
/// creates an iterator
async fn execute(&self, partition: usize) -> Result<SendableRecordBatchStream>;
/// Return a snapshot of the metrics collected during execution
fn metrics(&self) -> HashMap<String, SQLMetric> {
HashMap::new()
}
/// Format this `ExecutionPlan` to `f` in the specified type.
///
/// Should not include a newline
///
/// Note this function prints a placeholder by default to preserve
/// backwards compatibility.
fn fmt_as(&self, _t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result {
write!(f, "ExecutionPlan(PlaceHolder)")
}
}
/// Return a [wrapper](DisplayableExecutionPlan) around an
/// [`ExecutionPlan`] which can be displayed in various easier to
/// understand ways.
///
///
/// ```no_run as CubeStore disables coalesce batches.
/// use datafusion::prelude::*;
/// use datafusion::physical_plan::displayable;
///
/// // Hard code concurrency as it appears in the RepartitionExec output
/// let config = ExecutionConfig::new()
/// .with_concurrency(3);
/// let mut ctx = ExecutionContext::with_config(config);
///
/// // register the a table
/// ctx.register_csv("example", "tests/example.csv", CsvReadOptions::new()).unwrap();
///
/// // create a plan to run a SQL query
/// let plan = ctx
/// .create_logical_plan("SELECT a FROM example WHERE a < 5")
/// .unwrap();
/// let plan = ctx.optimize(&plan).unwrap();
/// let physical_plan = ctx.create_physical_plan(&plan).unwrap();
///
/// // Format using display string
/// let displayable_plan = displayable(physical_plan.as_ref());
/// let plan_string = format!("{}", displayable_plan.indent());
///
/// assert_eq!("ProjectionExec: expr=[a@0 as a]\
/// \n CoalesceBatchesExec: target_batch_size=4096\
/// \n FilterExec: a@0 < 5\
/// \n RepartitionExec: partitioning=RoundRobinBatch(3)\
/// \n CsvExec: source=Path(tests/example.csv: [tests/example.csv]), has_header=true",
/// plan_string.trim());
/// ```
///
pub fn displayable(plan: &dyn ExecutionPlan) -> DisplayableExecutionPlan<'_> {
DisplayableExecutionPlan::new(plan)
}
/// Visit all children of this plan, according to the order defined on `ExecutionPlanVisitor`.
// Note that this would be really nice if it were a method on
// ExecutionPlan, but it can not be because it takes a generic
// parameter and `ExecutionPlan` is a trait
pub fn accept<V: ExecutionPlanVisitor>(
plan: &dyn ExecutionPlan,
visitor: &mut V,
) -> std::result::Result<(), V::Error> {
visitor.pre_visit(plan)?;
for child in plan.children() {
visit_execution_plan(child.as_ref(), visitor)?;
}
visitor.post_visit(plan)?;
Ok(())
}
/// Trait that implements the [Visitor
/// pattern](https://en.wikipedia.org/wiki/Visitor_pattern) for a
/// depth first walk of `ExecutionPlan` nodes. `pre_visit` is called
/// before any children are visited, and then `post_visit` is called
/// after all children have been visited.
////
/// To use, define a struct that implements this trait and then invoke
/// ['accept'].
///
/// For example, for an execution plan that looks like:
///
/// ```text
/// ProjectionExec: #id
/// FilterExec: state = CO
/// CsvExec:
/// ```
///
/// The sequence of visit operations would be:
/// ```text
/// visitor.pre_visit(ProjectionExec)
/// visitor.pre_visit(FilterExec)
/// visitor.pre_visit(CsvExec)
/// visitor.post_visit(CsvExec)
/// visitor.post_visit(FilterExec)
/// visitor.post_visit(ProjectionExec)
/// ```
pub trait ExecutionPlanVisitor {
/// The type of error returned by this visitor
type Error;
/// Invoked on an `ExecutionPlan` plan before any of its child
/// inputs have been visited. If Ok(true) is returned, the
/// recursion continues. If Err(..) or Ok(false) are returned, the
/// recursion stops immediately and the error, if any, is returned
/// to `accept`
fn pre_visit(
&mut self,
plan: &dyn ExecutionPlan,
) -> std::result::Result<bool, Self::Error>;
/// Invoked on an `ExecutionPlan` plan *after* all of its child
/// inputs have been visited. The return value is handled the same
/// as the return value of `pre_visit`. The provided default
/// implementation returns `Ok(true)`.
fn post_visit(
&mut self,
_plan: &dyn ExecutionPlan,
) -> std::result::Result<bool, Self::Error> {
Ok(true)
}
}
/// Recursively calls `pre_visit` and `post_visit` for this node and
/// all of its children, as described on [`ExecutionPlanVisitor`]
pub fn visit_execution_plan<V: ExecutionPlanVisitor>(
plan: &dyn ExecutionPlan,
visitor: &mut V,
) -> std::result::Result<(), V::Error> {
visitor.pre_visit(plan)?;
for child in plan.children() {
visit_execution_plan(child.as_ref(), visitor)?;
}
visitor.post_visit(plan)?;
Ok(())
}
/// Recursively gateher all execution metrics from this plan and all of its input plans
pub fn plan_metrics(plan: Arc<dyn ExecutionPlan>) -> HashMap<String, SQLMetric> {
fn get_metrics_inner(
plan: &dyn ExecutionPlan,
mut metrics: HashMap<String, SQLMetric>,
) -> HashMap<String, SQLMetric> {
metrics.extend(plan.metrics().into_iter());
plan.children().into_iter().fold(metrics, |metrics, child| {
get_metrics_inner(child.as_ref(), metrics)
})
}
get_metrics_inner(plan.as_ref(), HashMap::new())
}
/// Execute the [ExecutionPlan] and collect the results in memory
pub async fn collect(plan: Arc<dyn ExecutionPlan>) -> Result<Vec<RecordBatch>> {
match plan.output_partitioning().partition_count() {
0 => Ok(vec![]),
1 => {
let it = plan.execute(0).await?;
common::collect(it).await
}
_ => {
// merge into a single partition
let plan = CoalescePartitionsExec::new(plan.clone());
// CoalescePartitionsExec must produce a single partition
assert_eq!(1, plan.output_partitioning().partition_count());
common::collect(plan.execute(0).await?).await
}
}
}
/// Execute the [ExecutionPlan] and collect the results in memory
pub async fn collect_partitioned(
plan: Arc<dyn ExecutionPlan>,
) -> Result<Vec<Vec<RecordBatch>>> {
match plan.output_partitioning().partition_count() {
0 => Ok(vec![]),
1 => {
let it = plan.execute(0).await?;
Ok(vec![common::collect(it).await?])
}
_ => {
let mut partitions = vec![];
for i in 0..plan.output_partitioning().partition_count() {
partitions.push(common::collect(plan.execute(i).await?).await?)
}
Ok(partitions)
}
}
}
/// Partitioning schemes supported by operators.
#[derive(Debug, Clone)]
pub enum Partitioning {
/// Allocate batches using a round-robin algorithm and the specified number of partitions
RoundRobinBatch(usize),
/// Allocate rows based on a hash of one of more expressions and the specified number of
/// partitions
Hash(Vec<Arc<dyn PhysicalExpr>>, usize),
/// Unknown partitioning scheme with a known number of partitions
UnknownPartitioning(usize),
}
impl Partitioning {
/// Returns the number of partitions in this partitioning scheme
pub fn partition_count(&self) -> usize {
use Partitioning::*;
match self {
RoundRobinBatch(n) => *n,
Hash(_, n) => *n,
UnknownPartitioning(n) => *n,
}
}
}
/// Distribution schemes
#[derive(Debug, Clone)]
pub enum Distribution {
/// Unspecified distribution
UnspecifiedDistribution,
/// A single partition is required
SinglePartition,
/// Requires children to be distributed in such a way that the same
/// values of the keys end up in the same partition
HashPartitioned(Vec<Arc<dyn PhysicalExpr>>),
}
/// Represents the result from an expression
#[derive(Debug, Clone)]
pub enum ColumnarValue {
/// Array of values
Array(ArrayRef),
/// A single value
Scalar(ScalarValue),
}
impl ColumnarValue {
#[allow(missing_docs)]
pub fn data_type(&self) -> DataType {
match self {
ColumnarValue::Array(array_value) => array_value.data_type().clone(),
ColumnarValue::Scalar(scalar_value) => scalar_value.get_datatype(),
}
}
/// Convert a columnar value into an ArrayRef
pub fn into_array(self, num_rows: usize) -> ArrayRef {
match self {
ColumnarValue::Array(array) => array,
ColumnarValue::Scalar(scalar) => scalar.to_array_of_size(num_rows),
}
}
}
/// Expression that can be evaluated against a RecordBatch
/// A Physical expression knows its type, nullability and how to evaluate itself.
pub trait PhysicalExpr: Send + Sync + Display + Debug {
/// Returns the physical expression as [`Any`](std::any::Any) so that it can be
/// downcast to a specific implementation.
fn as_any(&self) -> &dyn Any;
/// Get the data type of this expression, given the schema of the input
fn data_type(&self, input_schema: &Schema) -> Result<DataType>;
/// Determine whether this expression is nullable, given the schema of the input
fn nullable(&self, input_schema: &Schema) -> Result<bool>;
/// Evaluate an expression against a RecordBatch
fn evaluate(&self, batch: &RecordBatch) -> Result<ColumnarValue>;
}
/// An aggregate expression that:
/// * knows its resulting field
/// * knows how to create its accumulator
/// * knows its accumulator's state's field
/// * knows the expressions from whose its accumulator will receive values
pub trait AggregateExpr: Send + Sync + Debug {
/// Returns the aggregate expression as [`Any`](std::any::Any) so that it can be
/// downcast to a specific implementation.
fn as_any(&self) -> &dyn Any;
/// the field of the final result of this aggregation.
fn field(&self) -> Result<Field>;
/// the accumulator used to accumulate values from the expressions.
/// the accumulator expects the same number of arguments as `expressions` and must
/// return states with the same description as `state_fields`
fn create_accumulator(&self) -> Result<Box<dyn Accumulator>>;
/// Returns true if and only if create_groups_accumulator returns Ok(Some(_)) (if not an Err(_)).
fn uses_groups_accumulator(&self) -> bool {
return false;
}
/// the groups accumulator used to accumulate values from the expression. If this returns None,
/// create_accumulator must be used.
fn create_groups_accumulator(
&self,
) -> ArrowResult<Option<Box<dyn GroupsAccumulator>>> {
Ok(None)
}
/// the fields that encapsulate the Accumulator's state
/// the number of fields here equals the number of states that the accumulator contains
fn state_fields(&self) -> Result<Vec<Field>>;
/// expressions that are passed to the Accumulator.
/// Single-column aggregations such as `sum` return a single value, others (e.g. `cov`) return many.
fn expressions(&self) -> Vec<Arc<dyn PhysicalExpr>>;
/// Human readable name such as `"MIN(c2)"`. The default
/// implementation returns placeholder text.
fn name(&self) -> &str {
"AggregateExpr: default name"
}
}
/// A window expression that:
/// * knows its resulting field
pub trait WindowExpr: Send + Sync + Debug {
/// Returns the window expression as [`Any`](std::any::Any) so that it can be
/// downcast to a specific implementation.
fn as_any(&self) -> &dyn Any;
/// the field of the final result of this window function.
fn field(&self) -> Result<Field>;
/// Human readable name such as `"MIN(c2)"` or `"RANK()"`. The default
/// implementation returns placeholder text.
fn name(&self) -> &str {
"WindowExpr: default name"
}
/// expressions that are passed to the WindowAccumulator.
/// Functions which take a single input argument, such as `sum`, return a single [`Expr`],
/// others (e.g. `cov`) return many.
fn expressions(&self) -> Vec<Arc<dyn PhysicalExpr>>;
/// evaluate the window function arguments against the batch and return
/// array ref, normally the resulting vec is a single element one.
fn evaluate_args(&self, batch: &RecordBatch) -> Result<Vec<ArrayRef>> {
self.expressions()
.iter()
.map(|e| e.evaluate(batch))
.map(|r| r.map(|v| v.into_array(batch.num_rows())))
.collect()
}
/// evaluate the window function values against the batch
fn evaluate(&self, batch: &RecordBatch) -> Result<ArrayRef>;
/// evaluate the partition points given the sort columns; if the sort columns are
/// empty then the result will be a single element vec of the whole column rows.
fn evaluate_partition_points(
&self,
num_rows: usize,
partition_columns: &[SortColumn],
) -> Result<Vec<Range<usize>>> {
if partition_columns.is_empty() {
Ok(vec![Range {
start: 0,
end: num_rows,
}])
} else {
Ok(lexicographical_partition_ranges(partition_columns)
.map_err(DataFusionError::ArrowError)?
.collect::<Vec<_>>())
}
}
/// expressions that's from the window function's partition by clause, empty if absent
fn partition_by(&self) -> &[Arc<dyn PhysicalExpr>];
/// expressions that's from the window function's order by clause, empty if absent
fn order_by(&self) -> &[PhysicalSortExpr];
/// get partition columns that can be used for partitioning, empty if absent
fn partition_columns(&self, batch: &RecordBatch) -> Result<Vec<SortColumn>> {
self.partition_by()
.iter()
.map(|expr| {
PhysicalSortExpr {
expr: expr.clone(),
options: SortOptions::default(),
}
.evaluate_to_sort_column(batch)
})
.collect()
}
/// get sort columns that can be used for peer evaluation, empty if absent
fn sort_columns(&self, batch: &RecordBatch) -> Result<Vec<SortColumn>> {
let mut sort_columns = self.partition_columns(batch)?;
let order_by_columns = self
.order_by()
.iter()
.map(|e| e.evaluate_to_sort_column(batch))
.collect::<Result<Vec<SortColumn>>>()?;
sort_columns.extend(order_by_columns);
Ok(sort_columns)
}
}
/// An accumulator represents a stateful object that lives throughout the evaluation of multiple rows and
/// generically accumulates values.
///
/// An accumulator knows how to:
/// * update its state from inputs via `update`
/// * convert its internal state to a vector of scalar values
/// * update its state from multiple accumulators' states via `merge`
/// * compute the final value from its internal state via `evaluate`
pub trait Accumulator: Send + Sync + Debug {
/// Return accumulator to its initial state.
fn reset(&mut self);
/// Returns the state of the accumulator at the end of the accumulation.
// in the case of an average on which we track `sum` and `n`, this function should return a vector
// of two values, sum and n.
fn state(&self) -> Result<SmallVec<[ScalarValue; 2]>>;
/// updates the accumulator's state from a vector of scalars.
fn update(&mut self, values: &[ScalarValue]) -> Result<()>;
/// updates the accumulator's state from a vector of arrays.
fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> {
if values.is_empty() {
return Ok(());
};
(0..values[0].len()).try_for_each(|index| {
let v = values
.iter()
.map(|array| ScalarValue::try_from_array(array, index))
.collect::<Result<Vec<_>>>()?;
self.update(&v)
})
}
/// updates the accumulator's state from a vector of scalars.
fn merge(&mut self, states: &[ScalarValue]) -> Result<()>;
/// updates the accumulator's state from a vector of states.
fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> {
if states.is_empty() {
return Ok(());
};
(0..states[0].len()).try_for_each(|index| {
let v = states
.iter()
.map(|array| ScalarValue::try_from_array(array, index))
.collect::<Result<Vec<_>>>()?;
self.merge(&v)
})
}
/// returns its value based on its current state.
fn evaluate(&self) -> Result<ScalarValue>;
}
// Used for AggregateExpr implementations that still have uses_groups_accumulator being false.
impl Accumulator for Box<dyn Accumulator> {
fn reset(&mut self) {
self.as_mut().reset()
}
fn state(&self) -> Result<SmallVec<[ScalarValue; 2]>> {
self.as_ref().state()
}
fn update(&mut self, values: &[ScalarValue]) -> Result<()> {
self.as_mut().update(values)
}
fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> {
self.as_mut().update_batch(values)
}
fn merge(&mut self, states: &[ScalarValue]) -> Result<()> {
self.as_mut().merge(states)
}
fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> {
self.as_mut().merge_batch(states)
}
fn evaluate(&self) -> Result<ScalarValue> {
self.as_ref().evaluate()
}
}
pub mod aggregates;
pub mod array_expressions;
pub mod coalesce_batches;
pub mod coalesce_partitions;
pub mod common;
pub mod cross_join;
#[cfg(feature = "crypto_expressions")]
pub mod crypto_expressions;
pub mod csv;
pub mod datetime_expressions;
pub mod display;
pub mod distinct_expressions;
pub mod empty;
pub mod explain;
pub mod expressions;
pub mod filter;
pub mod functions;
pub mod group_scalar;
pub mod groups_accumulator;
pub mod groups_accumulator_adapter;
pub mod groups_accumulator_flat_adapter;
pub mod groups_accumulator_prim_op;
pub mod hash_aggregate;
pub mod hash_join;
pub mod hash_utils;
pub mod json;
pub mod limit;
pub mod math_expressions;
pub mod memory;
pub mod merge;
pub mod merge_join;
pub mod merge_sort;
pub mod null_state;
pub mod parquet;
pub mod planner;
pub mod projection;
#[cfg(feature = "regex_expressions")]
pub mod regex_expressions;
pub mod repartition;
pub mod skip;
pub mod sort;
pub mod sort_preserving_merge;
mod sorted_aggregate;
pub mod source;
pub mod string_expressions;
pub mod type_coercion;
pub mod udaf;
pub mod udf;
#[cfg(feature = "unicode_expressions")]
pub mod unicode_expressions;
pub mod union;
pub mod window_functions;
pub mod windows;