forked from apache/datafusion
-
Notifications
You must be signed in to change notification settings - Fork 0
Expand file tree
/
Copy pathtopk_aggregation.rs
More file actions
166 lines (147 loc) · 5.79 KB
/
topk_aggregation.rs
File metadata and controls
166 lines (147 loc) · 5.79 KB
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
// 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.
//! An optimizer rule that detects aggregate operations that could use a limited bucket count
use std::sync::Arc;
use datafusion_physical_plan::aggregates::AggregateExec;
use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec;
use datafusion_physical_plan::filter::FilterExec;
use datafusion_physical_plan::repartition::RepartitionExec;
use datafusion_physical_plan::sorts::sort::SortExec;
use datafusion_physical_plan::ExecutionPlan;
use arrow_schema::DataType;
use datafusion_common::config::ConfigOptions;
use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode};
use datafusion_common::Result;
use datafusion_physical_expr::expressions::Column;
use datafusion_physical_expr::PhysicalSortExpr;
use crate::PhysicalOptimizerRule;
use itertools::Itertools;
/// An optimizer rule that passes a `limit` hint to aggregations if the whole result is not needed
pub struct TopKAggregation {}
impl TopKAggregation {
/// Create a new `LimitAggregation`
pub fn new() -> Self {
Self {}
}
fn transform_agg(
aggr: &AggregateExec,
order: &PhysicalSortExpr,
limit: usize,
) -> Option<Arc<dyn ExecutionPlan>> {
// ensure the sort direction matches aggregate function
let (field, desc) = aggr.get_minmax_desc()?;
if desc != order.options.descending {
return None;
}
let group_key = aggr.group_expr().expr().iter().exactly_one().ok()?;
let kt = group_key.0.data_type(&aggr.input().schema()).ok()?;
if !kt.is_primitive() && kt != DataType::Utf8 {
return None;
}
if aggr.filter_expr().iter().any(|e| e.is_some()) {
return None;
}
// ensure the sort is on the same field as the aggregate output
let col = order.expr.as_any().downcast_ref::<Column>()?;
if col.name() != field.name() {
return None;
}
// We found what we want: clone, copy the limit down, and return modified node
let new_aggr = AggregateExec::try_new(
*aggr.mode(),
aggr.group_expr().clone(),
aggr.aggr_expr().to_vec(),
aggr.filter_expr().to_vec(),
Arc::clone(aggr.input()),
aggr.input_schema(),
)
.expect("Unable to copy Aggregate!")
.with_limit(Some(limit));
Some(Arc::new(new_aggr))
}
fn transform_sort(plan: &Arc<dyn ExecutionPlan>) -> Option<Arc<dyn ExecutionPlan>> {
let sort = plan.as_any().downcast_ref::<SortExec>()?;
let children = sort.children();
let child = children.into_iter().exactly_one().ok()?;
let order = sort.properties().output_ordering()?;
let order = order.iter().exactly_one().ok()?;
let limit = sort.fetch()?;
let is_cardinality_preserving = |plan: Arc<dyn ExecutionPlan>| {
plan.as_any()
.downcast_ref::<CoalesceBatchesExec>()
.is_some()
|| plan.as_any().downcast_ref::<RepartitionExec>().is_some()
|| plan.as_any().downcast_ref::<FilterExec>().is_some()
};
let mut cardinality_preserved = true;
let closure = |plan: Arc<dyn ExecutionPlan>| {
if !cardinality_preserved {
return Ok(Transformed::no(plan));
}
if let Some(aggr) = plan.as_any().downcast_ref::<AggregateExec>() {
// either we run into an Aggregate and transform it
match Self::transform_agg(aggr, order, limit) {
None => cardinality_preserved = false,
Some(plan) => return Ok(Transformed::yes(plan)),
}
} else {
// or we continue down whitelisted nodes of other types
if !is_cardinality_preserving(Arc::clone(&plan)) {
cardinality_preserved = false;
}
}
Ok(Transformed::no(plan))
};
let child = Arc::clone(child).transform_down(closure).data().ok()?;
let sort = SortExec::new(sort.expr().to_vec(), child)
.with_fetch(sort.fetch())
.with_preserve_partitioning(sort.preserve_partitioning());
Some(Arc::new(sort))
}
}
impl Default for TopKAggregation {
fn default() -> Self {
Self::new()
}
}
impl PhysicalOptimizerRule for TopKAggregation {
fn optimize(
&self,
plan: Arc<dyn ExecutionPlan>,
config: &ConfigOptions,
) -> Result<Arc<dyn ExecutionPlan>> {
if config.optimizer.enable_topk_aggregation {
plan.transform_down(|plan| {
Ok(if let Some(plan) = TopKAggregation::transform_sort(&plan) {
Transformed::yes(plan)
} else {
Transformed::no(plan)
})
})
.data()
} else {
Ok(plan)
}
}
fn name(&self) -> &str {
"LimitAggregation"
}
fn schema_check(&self) -> bool {
true
}
}
// see `aggregate.slt` for tests