datafusion_physical_optimizer/
optimizer.rs

1// Licensed to the Apache Software Foundation (ASF) under one
2// or more contributor license agreements.  See the NOTICE file
3// distributed with this work for additional information
4// regarding copyright ownership.  The ASF licenses this file
5// to you under the Apache License, Version 2.0 (the
6// "License"); you may not use this file except in compliance
7// with the License.  You may obtain a copy of the License at
8//
9//   http://www.apache.org/licenses/LICENSE-2.0
10//
11// Unless required by applicable law or agreed to in writing,
12// software distributed under the License is distributed on an
13// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
14// KIND, either express or implied.  See the License for the
15// specific language governing permissions and limitations
16// under the License.
17
18//! Physical optimizer traits
19
20use std::fmt::Debug;
21use std::sync::Arc;
22
23use crate::aggregate_statistics::AggregateStatistics;
24use crate::coalesce_batches::CoalesceBatches;
25use crate::combine_partial_final_agg::CombinePartialFinalAggregate;
26use crate::enforce_distribution::EnforceDistribution;
27use crate::enforce_sorting::EnforceSorting;
28use crate::ensure_coop::EnsureCooperative;
29use crate::filter_pushdown::FilterPushdown;
30use crate::join_selection::JoinSelection;
31use crate::limit_pushdown::LimitPushdown;
32use crate::limited_distinct_aggregation::LimitedDistinctAggregation;
33use crate::output_requirements::OutputRequirements;
34use crate::projection_pushdown::ProjectionPushdown;
35use crate::sanity_checker::SanityCheckPlan;
36use crate::topk_aggregation::TopKAggregation;
37use crate::update_aggr_exprs::OptimizeAggregateOrder;
38
39use crate::coalesce_async_exec_input::CoalesceAsyncExecInput;
40use crate::limit_pushdown_past_window::LimitPushPastWindows;
41use datafusion_common::config::ConfigOptions;
42use datafusion_common::Result;
43use datafusion_physical_plan::ExecutionPlan;
44
45/// `PhysicalOptimizerRule` transforms one ['ExecutionPlan'] into another which
46/// computes the same results, but in a potentially more efficient way.
47///
48/// Use [`SessionState::add_physical_optimizer_rule`] to register additional
49/// `PhysicalOptimizerRule`s.
50///
51/// [`SessionState::add_physical_optimizer_rule`]: https://docs.rs/datafusion/latest/datafusion/execution/session_state/struct.SessionState.html#method.add_physical_optimizer_rule
52pub trait PhysicalOptimizerRule: Debug {
53    /// Rewrite `plan` to an optimized form
54    fn optimize(
55        &self,
56        plan: Arc<dyn ExecutionPlan>,
57        config: &ConfigOptions,
58    ) -> Result<Arc<dyn ExecutionPlan>>;
59
60    /// A human readable name for this optimizer rule
61    fn name(&self) -> &str;
62
63    /// A flag to indicate whether the physical planner should validate that the rule will not
64    /// change the schema of the plan after the rewriting.
65    /// Some of the optimization rules might change the nullable properties of the schema
66    /// and should disable the schema check.
67    fn schema_check(&self) -> bool;
68}
69
70/// A rule-based physical optimizer.
71#[derive(Clone, Debug)]
72pub struct PhysicalOptimizer {
73    /// All rules to apply
74    pub rules: Vec<Arc<dyn PhysicalOptimizerRule + Send + Sync>>,
75}
76
77impl Default for PhysicalOptimizer {
78    fn default() -> Self {
79        Self::new()
80    }
81}
82
83impl PhysicalOptimizer {
84    /// Create a new optimizer using the recommended list of rules
85    pub fn new() -> Self {
86        let rules: Vec<Arc<dyn PhysicalOptimizerRule + Send + Sync>> = vec![
87            // If there is a output requirement of the query, make sure that
88            // this information is not lost across different rules during optimization.
89            Arc::new(OutputRequirements::new_add_mode()),
90            Arc::new(AggregateStatistics::new()),
91            // Statistics-based join selection will change the Auto mode to a real join implementation,
92            // like collect left, or hash join, or future sort merge join, which will influence the
93            // EnforceDistribution and EnforceSorting rules as they decide whether to add additional
94            // repartitioning and local sorting steps to meet distribution and ordering requirements.
95            // Therefore, it should run before EnforceDistribution and EnforceSorting.
96            Arc::new(JoinSelection::new()),
97            // The LimitedDistinctAggregation rule should be applied before the EnforceDistribution rule,
98            // as that rule may inject other operations in between the different AggregateExecs.
99            // Applying the rule early means only directly-connected AggregateExecs must be examined.
100            Arc::new(LimitedDistinctAggregation::new()),
101            // The FilterPushdown rule tries to push down filters as far as it can.
102            // For example, it will push down filtering from a `FilterExec` to `DataSourceExec`.
103            // Note that this does not push down dynamic filters (such as those created by a `SortExec` operator in TopK mode),
104            // those are handled by the later `FilterPushdown` rule.
105            // See `FilterPushdownPhase` for more details.
106            Arc::new(FilterPushdown::new()),
107            // The EnforceDistribution rule is for adding essential repartitioning to satisfy distribution
108            // requirements. Please make sure that the whole plan tree is determined before this rule.
109            // This rule increases parallelism if doing so is beneficial to the physical plan; i.e. at
110            // least one of the operators in the plan benefits from increased parallelism.
111            Arc::new(EnforceDistribution::new()),
112            // The CombinePartialFinalAggregate rule should be applied after the EnforceDistribution rule
113            Arc::new(CombinePartialFinalAggregate::new()),
114            // The EnforceSorting rule is for adding essential local sorting to satisfy the required
115            // ordering. Please make sure that the whole plan tree is determined before this rule.
116            // Note that one should always run this rule after running the EnforceDistribution rule
117            // as the latter may break local sorting requirements.
118            Arc::new(EnforceSorting::new()),
119            // Run once after the local sorting requirement is changed
120            Arc::new(OptimizeAggregateOrder::new()),
121            // TODO: `try_embed_to_hash_join` in the ProjectionPushdown rule would be block by the CoalesceBatches, so add it before CoalesceBatches. Maybe optimize it in the future.
122            Arc::new(ProjectionPushdown::new()),
123            // The CoalesceBatches rule will not influence the distribution and ordering of the
124            // whole plan tree. Therefore, to avoid influencing other rules, it should run last.
125            Arc::new(CoalesceBatches::new()),
126            Arc::new(CoalesceAsyncExecInput::new()),
127            // Remove the ancillary output requirement operator since we are done with the planning
128            // phase.
129            Arc::new(OutputRequirements::new_remove_mode()),
130            // The aggregation limiter will try to find situations where the accumulator count
131            // is not tied to the cardinality, i.e. when the output of the aggregation is passed
132            // into an `order by max(x) limit y`. In this case it will copy the limit value down
133            // to the aggregation, allowing it to use only y number of accumulators.
134            Arc::new(TopKAggregation::new()),
135            // Tries to push limits down through window functions, growing as appropriate
136            // This can possibly be combined with [LimitPushdown]
137            // It needs to come after [EnforceSorting]
138            Arc::new(LimitPushPastWindows::new()),
139            // The LimitPushdown rule tries to push limits down as far as possible,
140            // replacing operators with fetching variants, or adding limits
141            // past operators that support limit pushdown.
142            Arc::new(LimitPushdown::new()),
143            // The ProjectionPushdown rule tries to push projections towards
144            // the sources in the execution plan. As a result of this process,
145            // a projection can disappear if it reaches the source providers, and
146            // sequential projections can merge into one. Even if these two cases
147            // are not present, the load of executors such as join or union will be
148            // reduced by narrowing their input tables.
149            Arc::new(ProjectionPushdown::new()),
150            Arc::new(EnsureCooperative::new()),
151            // This FilterPushdown handles dynamic filters that may have references to the source ExecutionPlan.
152            // Therefore it should be run at the end of the optimization process since any changes to the plan may break the dynamic filter's references.
153            // See `FilterPushdownPhase` for more details.
154            Arc::new(FilterPushdown::new_post_optimization()),
155            // The SanityCheckPlan rule checks whether the order and
156            // distribution requirements of each node in the plan
157            // is satisfied. It will also reject non-runnable query
158            // plans that use pipeline-breaking operators on infinite
159            // input(s). The rule generates a diagnostic error
160            // message for invalid plans. It makes no changes to the
161            // given query plan; i.e. it only acts as a final
162            // gatekeeping rule.
163            Arc::new(SanityCheckPlan::new()),
164        ];
165
166        Self::with_rules(rules)
167    }
168
169    /// Create a new optimizer with the given rules
170    pub fn with_rules(rules: Vec<Arc<dyn PhysicalOptimizerRule + Send + Sync>>) -> Self {
171        Self { rules }
172    }
173}