datafusion_physical_optimizer/
aggregate_statistics.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
// 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.

//! Utilizing exact statistics from sources to avoid scanning data
use std::sync::Arc;

use datafusion_common::config::ConfigOptions;
use datafusion_common::scalar::ScalarValue;
use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode};
use datafusion_common::Result;
use datafusion_physical_plan::aggregates::AggregateExec;
use datafusion_physical_plan::placeholder_row::PlaceholderRowExec;
use datafusion_physical_plan::projection::ProjectionExec;
use datafusion_physical_plan::udaf::{AggregateFunctionExpr, StatisticsArgs};
use datafusion_physical_plan::{expressions, ExecutionPlan};

use crate::PhysicalOptimizerRule;

/// Optimizer that uses available statistics for aggregate functions
#[derive(Default, Debug)]
pub struct AggregateStatistics {}

impl AggregateStatistics {
    #[allow(missing_docs)]
    pub fn new() -> Self {
        Self {}
    }
}

impl PhysicalOptimizerRule for AggregateStatistics {
    fn optimize(
        &self,
        plan: Arc<dyn ExecutionPlan>,
        _config: &ConfigOptions,
    ) -> Result<Arc<dyn ExecutionPlan>> {
        if let Some(partial_agg_exec) = take_optimizable(&*plan) {
            let partial_agg_exec = partial_agg_exec
                .as_any()
                .downcast_ref::<AggregateExec>()
                .expect("take_optimizable() ensures that this is a AggregateExec");
            let stats = partial_agg_exec.input().statistics()?;
            let mut projections = vec![];
            for expr in partial_agg_exec.aggr_expr() {
                let field = expr.field();
                let args = expr.expressions();
                let statistics_args = StatisticsArgs {
                    statistics: &stats,
                    return_type: field.data_type(),
                    is_distinct: expr.is_distinct(),
                    exprs: args.as_slice(),
                };
                if let Some((optimizable_statistic, name)) =
                    take_optimizable_value_from_statistics(&statistics_args, expr)
                {
                    projections
                        .push((expressions::lit(optimizable_statistic), name.to_owned()));
                } else {
                    // TODO: we need all aggr_expr to be resolved (cf TODO fullres)
                    break;
                }
            }

            // TODO fullres: use statistics even if not all aggr_expr could be resolved
            if projections.len() == partial_agg_exec.aggr_expr().len() {
                // input can be entirely removed
                Ok(Arc::new(ProjectionExec::try_new(
                    projections,
                    Arc::new(PlaceholderRowExec::new(plan.schema())),
                )?))
            } else {
                plan.map_children(|child| {
                    self.optimize(child, _config).map(Transformed::yes)
                })
                .data()
            }
        } else {
            plan.map_children(|child| self.optimize(child, _config).map(Transformed::yes))
                .data()
        }
    }

    fn name(&self) -> &str {
        "aggregate_statistics"
    }

    /// This rule will change the nullable properties of the schema, disable the schema check.
    fn schema_check(&self) -> bool {
        false
    }
}

/// assert if the node passed as argument is a final `AggregateExec` node that can be optimized:
/// - its child (with possible intermediate layers) is a partial `AggregateExec` node
/// - they both have no grouping expression
///
/// If this is the case, return a ref to the partial `AggregateExec`, else `None`.
/// We would have preferred to return a casted ref to AggregateExec but the recursion requires
/// the `ExecutionPlan.children()` method that returns an owned reference.
fn take_optimizable(node: &dyn ExecutionPlan) -> Option<Arc<dyn ExecutionPlan>> {
    if let Some(final_agg_exec) = node.as_any().downcast_ref::<AggregateExec>() {
        if !final_agg_exec.mode().is_first_stage()
            && final_agg_exec.group_expr().is_empty()
        {
            let mut child = Arc::clone(final_agg_exec.input());
            loop {
                if let Some(partial_agg_exec) =
                    child.as_any().downcast_ref::<AggregateExec>()
                {
                    if partial_agg_exec.mode().is_first_stage()
                        && partial_agg_exec.group_expr().is_empty()
                        && partial_agg_exec.filter_expr().iter().all(|e| e.is_none())
                    {
                        return Some(child);
                    }
                }
                if let [childrens_child] = child.children().as_slice() {
                    child = Arc::clone(childrens_child);
                } else {
                    break;
                }
            }
        }
    }
    None
}

/// If this agg_expr is a max that is exactly defined in the statistics, return it.
fn take_optimizable_value_from_statistics(
    statistics_args: &StatisticsArgs,
    agg_expr: &AggregateFunctionExpr,
) -> Option<(ScalarValue, String)> {
    let value = agg_expr.fun().value_from_stats(statistics_args);
    value.map(|val| (val, agg_expr.name().to_string()))
}

#[cfg(test)]
mod tests {
    use crate::aggregate_statistics::AggregateStatistics;
    use crate::PhysicalOptimizerRule;
    use datafusion_common::config::ConfigOptions;
    use datafusion_common::utils::expr::COUNT_STAR_EXPANSION;
    use datafusion_execution::TaskContext;
    use datafusion_functions_aggregate::count::count_udaf;
    use datafusion_physical_expr::aggregate::AggregateExprBuilder;
    use datafusion_physical_expr::PhysicalExpr;
    use datafusion_physical_plan::aggregates::AggregateExec;
    use datafusion_physical_plan::projection::ProjectionExec;
    use datafusion_physical_plan::udaf::AggregateFunctionExpr;
    use datafusion_physical_plan::ExecutionPlan;
    use std::sync::Arc;

    use datafusion_common::Result;
    use datafusion_expr_common::operator::Operator;

    use datafusion_physical_plan::aggregates::PhysicalGroupBy;
    use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec;
    use datafusion_physical_plan::common;
    use datafusion_physical_plan::filter::FilterExec;
    use datafusion_physical_plan::memory::MemoryExec;

    use arrow::array::Int32Array;
    use arrow::datatypes::{DataType, Field, Schema};
    use arrow::record_batch::RecordBatch;
    use datafusion_common::cast::as_int64_array;
    use datafusion_physical_expr::expressions::{self, cast};
    use datafusion_physical_plan::aggregates::AggregateMode;

    /// Describe the type of aggregate being tested
    pub enum TestAggregate {
        /// Testing COUNT(*) type aggregates
        CountStar,

        /// Testing for COUNT(column) aggregate
        ColumnA(Arc<Schema>),
    }

    impl TestAggregate {
        /// Create a new COUNT(*) aggregate
        pub fn new_count_star() -> Self {
            Self::CountStar
        }

        /// Create a new COUNT(column) aggregate
        pub fn new_count_column(schema: &Arc<Schema>) -> Self {
            Self::ColumnA(Arc::clone(schema))
        }

        /// Return appropriate expr depending if COUNT is for col or table (*)
        pub fn count_expr(&self, schema: &Schema) -> AggregateFunctionExpr {
            AggregateExprBuilder::new(count_udaf(), vec![self.column()])
                .schema(Arc::new(schema.clone()))
                .alias(self.column_name())
                .build()
                .unwrap()
        }

        /// what argument would this aggregate need in the plan?
        fn column(&self) -> Arc<dyn PhysicalExpr> {
            match self {
                Self::CountStar => expressions::lit(COUNT_STAR_EXPANSION),
                Self::ColumnA(s) => expressions::col("a", s).unwrap(),
            }
        }

        /// What name would this aggregate produce in a plan?
        pub fn column_name(&self) -> &'static str {
            match self {
                Self::CountStar => "COUNT(*)",
                Self::ColumnA(_) => "COUNT(a)",
            }
        }

        /// What is the expected count?
        pub fn expected_count(&self) -> i64 {
            match self {
                TestAggregate::CountStar => 3,
                TestAggregate::ColumnA(_) => 2,
            }
        }
    }

    /// Mock data using a MemoryExec which has an exact count statistic
    fn mock_data() -> Result<Arc<MemoryExec>> {
        let schema = Arc::new(Schema::new(vec![
            Field::new("a", DataType::Int32, true),
            Field::new("b", DataType::Int32, true),
        ]));

        let batch = RecordBatch::try_new(
            Arc::clone(&schema),
            vec![
                Arc::new(Int32Array::from(vec![Some(1), Some(2), None])),
                Arc::new(Int32Array::from(vec![Some(4), None, Some(6)])),
            ],
        )?;

        Ok(Arc::new(MemoryExec::try_new(
            &[vec![batch]],
            Arc::clone(&schema),
            None,
        )?))
    }

    /// Checks that the count optimization was applied and we still get the right result
    async fn assert_count_optim_success(
        plan: AggregateExec,
        agg: TestAggregate,
    ) -> Result<()> {
        let task_ctx = Arc::new(TaskContext::default());
        let plan: Arc<dyn ExecutionPlan> = Arc::new(plan);

        let config = ConfigOptions::new();
        let optimized =
            AggregateStatistics::new().optimize(Arc::clone(&plan), &config)?;

        // A ProjectionExec is a sign that the count optimization was applied
        assert!(optimized.as_any().is::<ProjectionExec>());

        // run both the optimized and nonoptimized plan
        let optimized_result =
            common::collect(optimized.execute(0, Arc::clone(&task_ctx))?).await?;
        let nonoptimized_result = common::collect(plan.execute(0, task_ctx)?).await?;
        assert_eq!(optimized_result.len(), nonoptimized_result.len());

        //  and validate the results are the same and expected
        assert_eq!(optimized_result.len(), 1);
        check_batch(optimized_result.into_iter().next().unwrap(), &agg);
        // check the non optimized one too to ensure types and names remain the same
        assert_eq!(nonoptimized_result.len(), 1);
        check_batch(nonoptimized_result.into_iter().next().unwrap(), &agg);

        Ok(())
    }

    fn check_batch(batch: RecordBatch, agg: &TestAggregate) {
        let schema = batch.schema();
        let fields = schema.fields();
        assert_eq!(fields.len(), 1);

        let field = &fields[0];
        assert_eq!(field.name(), agg.column_name());
        assert_eq!(field.data_type(), &DataType::Int64);
        // note that nullabiolity differs

        assert_eq!(
            as_int64_array(batch.column(0)).unwrap().values(),
            &[agg.expected_count()]
        );
    }

    #[tokio::test]
    async fn test_count_partial_direct_child() -> Result<()> {
        // basic test case with the aggregation applied on a source with exact statistics
        let source = mock_data()?;
        let schema = source.schema();
        let agg = TestAggregate::new_count_star();

        let partial_agg = AggregateExec::try_new(
            AggregateMode::Partial,
            PhysicalGroupBy::default(),
            vec![Arc::new(agg.count_expr(&schema))],
            vec![None],
            source,
            Arc::clone(&schema),
        )?;

        let final_agg = AggregateExec::try_new(
            AggregateMode::Final,
            PhysicalGroupBy::default(),
            vec![Arc::new(agg.count_expr(&schema))],
            vec![None],
            Arc::new(partial_agg),
            Arc::clone(&schema),
        )?;

        assert_count_optim_success(final_agg, agg).await?;

        Ok(())
    }

    #[tokio::test]
    async fn test_count_partial_with_nulls_direct_child() -> Result<()> {
        // basic test case with the aggregation applied on a source with exact statistics
        let source = mock_data()?;
        let schema = source.schema();
        let agg = TestAggregate::new_count_column(&schema);

        let partial_agg = AggregateExec::try_new(
            AggregateMode::Partial,
            PhysicalGroupBy::default(),
            vec![Arc::new(agg.count_expr(&schema))],
            vec![None],
            source,
            Arc::clone(&schema),
        )?;

        let final_agg = AggregateExec::try_new(
            AggregateMode::Final,
            PhysicalGroupBy::default(),
            vec![Arc::new(agg.count_expr(&schema))],
            vec![None],
            Arc::new(partial_agg),
            Arc::clone(&schema),
        )?;

        assert_count_optim_success(final_agg, agg).await?;

        Ok(())
    }

    #[tokio::test]
    async fn test_count_partial_indirect_child() -> Result<()> {
        let source = mock_data()?;
        let schema = source.schema();
        let agg = TestAggregate::new_count_star();

        let partial_agg = AggregateExec::try_new(
            AggregateMode::Partial,
            PhysicalGroupBy::default(),
            vec![Arc::new(agg.count_expr(&schema))],
            vec![None],
            source,
            Arc::clone(&schema),
        )?;

        // We introduce an intermediate optimization step between the partial and final aggregtator
        let coalesce = CoalescePartitionsExec::new(Arc::new(partial_agg));

        let final_agg = AggregateExec::try_new(
            AggregateMode::Final,
            PhysicalGroupBy::default(),
            vec![Arc::new(agg.count_expr(&schema))],
            vec![None],
            Arc::new(coalesce),
            Arc::clone(&schema),
        )?;

        assert_count_optim_success(final_agg, agg).await?;

        Ok(())
    }

    #[tokio::test]
    async fn test_count_partial_with_nulls_indirect_child() -> Result<()> {
        let source = mock_data()?;
        let schema = source.schema();
        let agg = TestAggregate::new_count_column(&schema);

        let partial_agg = AggregateExec::try_new(
            AggregateMode::Partial,
            PhysicalGroupBy::default(),
            vec![Arc::new(agg.count_expr(&schema))],
            vec![None],
            source,
            Arc::clone(&schema),
        )?;

        // We introduce an intermediate optimization step between the partial and final aggregtator
        let coalesce = CoalescePartitionsExec::new(Arc::new(partial_agg));

        let final_agg = AggregateExec::try_new(
            AggregateMode::Final,
            PhysicalGroupBy::default(),
            vec![Arc::new(agg.count_expr(&schema))],
            vec![None],
            Arc::new(coalesce),
            Arc::clone(&schema),
        )?;

        assert_count_optim_success(final_agg, agg).await?;

        Ok(())
    }

    #[tokio::test]
    async fn test_count_inexact_stat() -> Result<()> {
        let source = mock_data()?;
        let schema = source.schema();
        let agg = TestAggregate::new_count_star();

        // adding a filter makes the statistics inexact
        let filter = Arc::new(FilterExec::try_new(
            expressions::binary(
                expressions::col("a", &schema)?,
                Operator::Gt,
                cast(expressions::lit(1u32), &schema, DataType::Int32)?,
                &schema,
            )?,
            source,
        )?);

        let partial_agg = AggregateExec::try_new(
            AggregateMode::Partial,
            PhysicalGroupBy::default(),
            vec![Arc::new(agg.count_expr(&schema))],
            vec![None],
            filter,
            Arc::clone(&schema),
        )?;

        let final_agg = AggregateExec::try_new(
            AggregateMode::Final,
            PhysicalGroupBy::default(),
            vec![Arc::new(agg.count_expr(&schema))],
            vec![None],
            Arc::new(partial_agg),
            Arc::clone(&schema),
        )?;

        let conf = ConfigOptions::new();
        let optimized =
            AggregateStatistics::new().optimize(Arc::new(final_agg), &conf)?;

        // check that the original ExecutionPlan was not replaced
        assert!(optimized.as_any().is::<AggregateExec>());

        Ok(())
    }

    #[tokio::test]
    async fn test_count_with_nulls_inexact_stat() -> Result<()> {
        let source = mock_data()?;
        let schema = source.schema();
        let agg = TestAggregate::new_count_column(&schema);

        // adding a filter makes the statistics inexact
        let filter = Arc::new(FilterExec::try_new(
            expressions::binary(
                expressions::col("a", &schema)?,
                Operator::Gt,
                cast(expressions::lit(1u32), &schema, DataType::Int32)?,
                &schema,
            )?,
            source,
        )?);

        let partial_agg = AggregateExec::try_new(
            AggregateMode::Partial,
            PhysicalGroupBy::default(),
            vec![Arc::new(agg.count_expr(&schema))],
            vec![None],
            filter,
            Arc::clone(&schema),
        )?;

        let final_agg = AggregateExec::try_new(
            AggregateMode::Final,
            PhysicalGroupBy::default(),
            vec![Arc::new(agg.count_expr(&schema))],
            vec![None],
            Arc::new(partial_agg),
            Arc::clone(&schema),
        )?;

        let conf = ConfigOptions::new();
        let optimized =
            AggregateStatistics::new().optimize(Arc::new(final_agg), &conf)?;

        // check that the original ExecutionPlan was not replaced
        assert!(optimized.as_any().is::<AggregateExec>());

        Ok(())
    }
}