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::join_selection::JoinSelection;
29use crate::limit_pushdown::LimitPushdown;
30use crate::limited_distinct_aggregation::LimitedDistinctAggregation;
31use crate::output_requirements::OutputRequirements;
32use crate::projection_pushdown::ProjectionPushdown;
33use crate::sanity_checker::SanityCheckPlan;
34use crate::topk_aggregation::TopKAggregation;
35use crate::update_aggr_exprs::OptimizeAggregateOrder;
36
37use datafusion_common::config::ConfigOptions;
38use datafusion_common::Result;
39use datafusion_physical_plan::ExecutionPlan;
40
41/// `PhysicalOptimizerRule` transforms one ['ExecutionPlan'] into another which
42/// computes the same results, but in a potentially more efficient way.
43///
44/// Use [`SessionState::add_physical_optimizer_rule`] to register additional
45/// `PhysicalOptimizerRule`s.
46///
47/// [`SessionState::add_physical_optimizer_rule`]: https://docs.rs/datafusion/latest/datafusion/execution/session_state/struct.SessionState.html#method.add_physical_optimizer_rule
48pub trait PhysicalOptimizerRule: Debug {
49    /// Rewrite `plan` to an optimized form
50    fn optimize(
51        &self,
52        plan: Arc<dyn ExecutionPlan>,
53        config: &ConfigOptions,
54    ) -> Result<Arc<dyn ExecutionPlan>>;
55
56    /// A human readable name for this optimizer rule
57    fn name(&self) -> &str;
58
59    /// A flag to indicate whether the physical planner should valid the rule will not
60    /// change the schema of the plan after the rewriting.
61    /// Some of the optimization rules might change the nullable properties of the schema
62    /// and should disable the schema check.
63    fn schema_check(&self) -> bool;
64}
65
66/// A rule-based physical optimizer.
67#[derive(Clone, Debug)]
68pub struct PhysicalOptimizer {
69    /// All rules to apply
70    pub rules: Vec<Arc<dyn PhysicalOptimizerRule + Send + Sync>>,
71}
72
73impl Default for PhysicalOptimizer {
74    fn default() -> Self {
75        Self::new()
76    }
77}
78
79impl PhysicalOptimizer {
80    /// Create a new optimizer using the recommended list of rules
81    pub fn new() -> Self {
82        let rules: Vec<Arc<dyn PhysicalOptimizerRule + Send + Sync>> = vec![
83            // If there is a output requirement of the query, make sure that
84            // this information is not lost across different rules during optimization.
85            Arc::new(OutputRequirements::new_add_mode()),
86            Arc::new(AggregateStatistics::new()),
87            // Statistics-based join selection will change the Auto mode to a real join implementation,
88            // like collect left, or hash join, or future sort merge join, which will influence the
89            // EnforceDistribution and EnforceSorting rules as they decide whether to add additional
90            // repartitioning and local sorting steps to meet distribution and ordering requirements.
91            // Therefore, it should run before EnforceDistribution and EnforceSorting.
92            Arc::new(JoinSelection::new()),
93            // The LimitedDistinctAggregation rule should be applied before the EnforceDistribution rule,
94            // as that rule may inject other operations in between the different AggregateExecs.
95            // Applying the rule early means only directly-connected AggregateExecs must be examined.
96            Arc::new(LimitedDistinctAggregation::new()),
97            // The EnforceDistribution rule is for adding essential repartitioning to satisfy distribution
98            // requirements. Please make sure that the whole plan tree is determined before this rule.
99            // This rule increases parallelism if doing so is beneficial to the physical plan; i.e. at
100            // least one of the operators in the plan benefits from increased parallelism.
101            Arc::new(EnforceDistribution::new()),
102            // The CombinePartialFinalAggregate rule should be applied after the EnforceDistribution rule
103            Arc::new(CombinePartialFinalAggregate::new()),
104            // The EnforceSorting rule is for adding essential local sorting to satisfy the required
105            // ordering. Please make sure that the whole plan tree is determined before this rule.
106            // Note that one should always run this rule after running the EnforceDistribution rule
107            // as the latter may break local sorting requirements.
108            Arc::new(EnforceSorting::new()),
109            // Run once after the local sorting requirement is changed
110            Arc::new(OptimizeAggregateOrder::new()),
111            // 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.
112            Arc::new(ProjectionPushdown::new()),
113            // The CoalesceBatches rule will not influence the distribution and ordering of the
114            // whole plan tree. Therefore, to avoid influencing other rules, it should run last.
115            Arc::new(CoalesceBatches::new()),
116            // Remove the ancillary output requirement operator since we are done with the planning
117            // phase.
118            Arc::new(OutputRequirements::new_remove_mode()),
119            // The aggregation limiter will try to find situations where the accumulator count
120            // is not tied to the cardinality, i.e. when the output of the aggregation is passed
121            // into an `order by max(x) limit y`. In this case it will copy the limit value down
122            // to the aggregation, allowing it to use only y number of accumulators.
123            Arc::new(TopKAggregation::new()),
124            // The ProjectionPushdown rule tries to push projections towards
125            // the sources in the execution plan. As a result of this process,
126            // a projection can disappear if it reaches the source providers, and
127            // sequential projections can merge into one. Even if these two cases
128            // are not present, the load of executors such as join or union will be
129            // reduced by narrowing their input tables.
130            Arc::new(ProjectionPushdown::new()),
131            // The LimitPushdown rule tries to push limits down as far as possible,
132            // replacing operators with fetching variants, or adding limits
133            // past operators that support limit pushdown.
134            Arc::new(LimitPushdown::new()),
135            // The SanityCheckPlan rule checks whether the order and
136            // distribution requirements of each node in the plan
137            // is satisfied. It will also reject non-runnable query
138            // plans that use pipeline-breaking operators on infinite
139            // input(s). The rule generates a diagnostic error
140            // message for invalid plans. It makes no changes to the
141            // given query plan; i.e. it only acts as a final
142            // gatekeeping rule.
143            Arc::new(SanityCheckPlan::new()),
144        ];
145
146        Self::with_rules(rules)
147    }
148
149    /// Create a new optimizer with the given rules
150    pub fn with_rules(rules: Vec<Arc<dyn PhysicalOptimizerRule + Send + Sync>>) -> Self {
151        Self { rules }
152    }
153}