diff --git a/datafusion/optimizer/src/analyzer/mod.rs b/datafusion/optimizer/src/analyzer/mod.rs index ae61aea997b70..cf0d74ebce0ea 100644 --- a/datafusion/optimizer/src/analyzer/mod.rs +++ b/datafusion/optimizer/src/analyzer/mod.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +//! [`Analyzer`] and [`AnalyzerRule`] use std::sync::Arc; use log::debug; diff --git a/datafusion/optimizer/src/decorrelate.rs b/datafusion/optimizer/src/decorrelate.rs index 12e84a63ea150..dbcf02b26ba66 100644 --- a/datafusion/optimizer/src/decorrelate.rs +++ b/datafusion/optimizer/src/decorrelate.rs @@ -15,6 +15,8 @@ // specific language governing permissions and limitations // under the License. +//! [`PullUpCorrelatedExpr`] converts correlated subqueries to `Joins` + use std::collections::{BTreeSet, HashMap}; use std::ops::Deref; @@ -31,8 +33,11 @@ use datafusion_expr::utils::{conjunction, find_join_exprs, split_conjunction}; use datafusion_expr::{expr, EmptyRelation, Expr, LogicalPlan, LogicalPlanBuilder}; use datafusion_physical_expr::execution_props::ExecutionProps; -/// This struct rewrite the sub query plan by pull up the correlated expressions(contains outer reference columns) from the inner subquery's 'Filter'. -/// It adds the inner reference columns to the 'Projection' or 'Aggregate' of the subquery if they are missing, so that they can be evaluated by the parent operator as the join condition. +/// This struct rewrite the sub query plan by pull up the correlated +/// expressions(contains outer reference columns) from the inner subquery's +/// 'Filter'. It adds the inner reference columns to the 'Projection' or +/// 'Aggregate' of the subquery if they are missing, so that they can be +/// evaluated by the parent operator as the join condition. pub struct PullUpCorrelatedExpr { pub join_filters: Vec, // mapping from the plan to its holding correlated columns @@ -54,7 +59,9 @@ pub struct PullUpCorrelatedExpr { /// This is used to handle the Count bug pub const UN_MATCHED_ROW_INDICATOR: &str = "__always_true"; -/// Mapping from expr display name to its evaluation result on empty record batch (for example: 'count(*)' is 'ScalarValue(0)', 'count(*) + 2' is 'ScalarValue(2)') +/// Mapping from expr display name to its evaluation result on empty record +/// batch (for example: 'count(*)' is 'ScalarValue(0)', 'count(*) + 2' is +/// 'ScalarValue(2)') pub type ExprResultMap = HashMap; impl TreeNodeRewriter for PullUpCorrelatedExpr { diff --git a/datafusion/optimizer/src/decorrelate_predicate_subquery.rs b/datafusion/optimizer/src/decorrelate_predicate_subquery.rs index b94cf37c5c12b..019e7507b1228 100644 --- a/datafusion/optimizer/src/decorrelate_predicate_subquery.rs +++ b/datafusion/optimizer/src/decorrelate_predicate_subquery.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +//! [`DecorrelatePredicateSubquery`] converts `IN`/`EXISTS` subquery predicates to `SEMI`/`ANTI` joins use std::collections::BTreeSet; use std::ops::Deref; use std::sync::Arc; diff --git a/datafusion/optimizer/src/eliminate_cross_join.rs b/datafusion/optimizer/src/eliminate_cross_join.rs index 7f65690a4a7cb..18a9c05b9dc65 100644 --- a/datafusion/optimizer/src/eliminate_cross_join.rs +++ b/datafusion/optimizer/src/eliminate_cross_join.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! Optimizer rule to eliminate cross join to inner join if join predicates are available in filters. +//! [`EliminateCrossJoin`] converts `CROSS JOIN` to `INNER JOIN` if join predicates are available. use std::collections::HashSet; use std::sync::Arc; diff --git a/datafusion/optimizer/src/eliminate_duplicated_expr.rs b/datafusion/optimizer/src/eliminate_duplicated_expr.rs index de05717a72e27..349d4d8878e02 100644 --- a/datafusion/optimizer/src/eliminate_duplicated_expr.rs +++ b/datafusion/optimizer/src/eliminate_duplicated_expr.rs @@ -15,6 +15,8 @@ // specific language governing permissions and limitations // under the License. +//! [`EliminateDuplicatedExpr`] Removes redundant expressions + use crate::optimizer::ApplyOrder; use crate::{OptimizerConfig, OptimizerRule}; use datafusion_common::Result; diff --git a/datafusion/optimizer/src/eliminate_filter.rs b/datafusion/optimizer/src/eliminate_filter.rs index fea14342ca774..9411dc192bebf 100644 --- a/datafusion/optimizer/src/eliminate_filter.rs +++ b/datafusion/optimizer/src/eliminate_filter.rs @@ -15,9 +15,8 @@ // specific language governing permissions and limitations // under the License. -//! Optimizer rule to replace `where false or null` on a plan with an empty relation. -//! This saves time in planning and executing the query. -//! Note that this rule should be applied after simplify expressions optimizer rule. +//! [`EliminateFilter`] replaces `where false` or `where null` with an empty relation. + use crate::optimizer::ApplyOrder; use datafusion_common::{Result, ScalarValue}; use datafusion_expr::{ @@ -27,7 +26,11 @@ use datafusion_expr::{ use crate::{OptimizerConfig, OptimizerRule}; -/// Optimization rule that eliminate the scalar value (true/false/null) filter with an [LogicalPlan::EmptyRelation] +/// Optimization rule that eliminate the scalar value (true/false/null) filter +/// with an [LogicalPlan::EmptyRelation] +/// +/// This saves time in planning and executing the query. +/// Note that this rule should be applied after simplify expressions optimizer rule. #[derive(Default)] pub struct EliminateFilter; diff --git a/datafusion/optimizer/src/eliminate_join.rs b/datafusion/optimizer/src/eliminate_join.rs index 0dbebcc8a0519..e685229c61b26 100644 --- a/datafusion/optimizer/src/eliminate_join.rs +++ b/datafusion/optimizer/src/eliminate_join.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +//! [`EliminateJoin`] rewrites `INNER JOIN` with `true`/`null` use crate::optimizer::ApplyOrder; use crate::{OptimizerConfig, OptimizerRule}; use datafusion_common::{Result, ScalarValue}; @@ -24,7 +25,7 @@ use datafusion_expr::{ CrossJoin, Expr, }; -/// Eliminates joins when inner join condition is false. +/// Eliminates joins when join condition is false. /// Replaces joins when inner join condition is true with a cross join. #[derive(Default)] pub struct EliminateJoin; diff --git a/datafusion/optimizer/src/eliminate_limit.rs b/datafusion/optimizer/src/eliminate_limit.rs index 4386253740aaa..fb5d0d17b839a 100644 --- a/datafusion/optimizer/src/eliminate_limit.rs +++ b/datafusion/optimizer/src/eliminate_limit.rs @@ -15,18 +15,19 @@ // specific language governing permissions and limitations // under the License. -//! Optimizer rule to replace `LIMIT 0` or -//! `LIMIT whose ancestor LIMIT's skip is greater than or equal to current's fetch` -//! on a plan with an empty relation. -//! This rule also removes OFFSET 0 from the [LogicalPlan] -//! This saves time in planning and executing the query. +//! [`EliminateLimit`] eliminates `LIMIT` when possible use crate::optimizer::ApplyOrder; use crate::{OptimizerConfig, OptimizerRule}; use datafusion_common::Result; use datafusion_expr::logical_plan::{EmptyRelation, LogicalPlan}; -/// Optimization rule that eliminate LIMIT 0 or useless LIMIT(skip:0, fetch:None). -/// It can cooperate with `propagate_empty_relation` and `limit_push_down`. +/// Optimizer rule to replace `LIMIT 0` or `LIMIT` whose ancestor LIMIT's skip is +/// greater than or equal to current's fetch +/// +/// It can cooperate with `propagate_empty_relation` and `limit_push_down`. on a +/// plan with an empty relation. +/// +/// This rule also removes OFFSET 0 from the [LogicalPlan] #[derive(Default)] pub struct EliminateLimit; diff --git a/datafusion/optimizer/src/eliminate_nested_union.rs b/datafusion/optimizer/src/eliminate_nested_union.rs index 5771ea2e19a29..924a0853418cd 100644 --- a/datafusion/optimizer/src/eliminate_nested_union.rs +++ b/datafusion/optimizer/src/eliminate_nested_union.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! Optimizer rule to replace nested unions to single union. +//! [`EliminateNestedUnion`]: flattens nested `Union` to a single `Union` use crate::optimizer::ApplyOrder; use crate::{OptimizerConfig, OptimizerRule}; use datafusion_common::Result; diff --git a/datafusion/optimizer/src/eliminate_one_union.rs b/datafusion/optimizer/src/eliminate_one_union.rs index 70ee490346ffb..63c3e789daa67 100644 --- a/datafusion/optimizer/src/eliminate_one_union.rs +++ b/datafusion/optimizer/src/eliminate_one_union.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! Optimizer rule to eliminate one union. +//! [`EliminateOneUnion`] eliminates single element `Union` use crate::{OptimizerConfig, OptimizerRule}; use datafusion_common::Result; use datafusion_expr::logical_plan::{LogicalPlan, Union}; diff --git a/datafusion/optimizer/src/eliminate_outer_join.rs b/datafusion/optimizer/src/eliminate_outer_join.rs index 56a4a76987f75..a004da2bff19b 100644 --- a/datafusion/optimizer/src/eliminate_outer_join.rs +++ b/datafusion/optimizer/src/eliminate_outer_join.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! Optimizer rule to eliminate left/right/full join to inner join if possible. +//! [`EliminateOuterJoin`] converts `LEFT/RIGHT/FULL` joins to `INNER` joins use crate::{OptimizerConfig, OptimizerRule}; use datafusion_common::{Column, DFSchema, Result}; use datafusion_expr::logical_plan::{Join, JoinType, LogicalPlan}; diff --git a/datafusion/optimizer/src/extract_equijoin_predicate.rs b/datafusion/optimizer/src/extract_equijoin_predicate.rs index 24664d57c38d8..4cfcd07b47d93 100644 --- a/datafusion/optimizer/src/extract_equijoin_predicate.rs +++ b/datafusion/optimizer/src/extract_equijoin_predicate.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! [`ExtractEquijoinPredicate`] rule that extracts equijoin predicates +//! [`ExtractEquijoinPredicate`] identifies equality join (equijoin) predicates use crate::optimizer::ApplyOrder; use crate::{OptimizerConfig, OptimizerRule}; use datafusion_common::DFSchema; diff --git a/datafusion/optimizer/src/filter_null_join_keys.rs b/datafusion/optimizer/src/filter_null_join_keys.rs index 95cd8a9fd36ca..16039b182bb20 100644 --- a/datafusion/optimizer/src/filter_null_join_keys.rs +++ b/datafusion/optimizer/src/filter_null_join_keys.rs @@ -15,10 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! The FilterNullJoinKeys rule will identify inner joins with equi-join conditions -//! where the join key is nullable on one side and non-nullable on the other side -//! and then insert an `IsNotNull` filter on the nullable side since null values -//! can never match. +//! [`FilterNullJoinKeys`] adds filters to join inputs when input isn't nullable use crate::optimizer::ApplyOrder; use crate::{OptimizerConfig, OptimizerRule}; diff --git a/datafusion/optimizer/src/lib.rs b/datafusion/optimizer/src/lib.rs index b54facc5d6825..f1f49727c39c2 100644 --- a/datafusion/optimizer/src/lib.rs +++ b/datafusion/optimizer/src/lib.rs @@ -15,6 +15,19 @@ // specific language governing permissions and limitations // under the License. +//! # DataFusion Optimizer +//! +//! Contains rules for rewriting [`LogicalPlan`]s +//! +//! 1. [`Analyzer`] applies [`AnalyzerRule`]s to transform `LogicalPlan`s +//! to make the plan valid prior to the rest of the DataFusion optimization +//! process (for example, [`TypeCoercion`]). +//! +//! 2. [`Optimizer`] applies [`OptimizerRule`]s to transform `LogicalPlan`s +//! into equivalent, but more efficient plans. +//! +//! [`LogicalPlan`]: datafusion_expr::LogicalPlan +//! [`TypeCoercion`]: analyzer::type_coercion::TypeCoercion pub mod analyzer; pub mod common_subexpr_eliminate; pub mod decorrelate; @@ -46,7 +59,8 @@ pub mod utils; #[cfg(test)] pub mod test; -pub use optimizer::{OptimizerConfig, OptimizerContext, OptimizerRule}; +pub use analyzer::{Analyzer, AnalyzerRule}; +pub use optimizer::{Optimizer, OptimizerConfig, OptimizerContext, OptimizerRule}; pub use utils::optimize_children; mod plan_signature; diff --git a/datafusion/optimizer/src/optimize_projections.rs b/datafusion/optimizer/src/optimize_projections.rs index c40a9bb704ebf..147702cc04411 100644 --- a/datafusion/optimizer/src/optimize_projections.rs +++ b/datafusion/optimizer/src/optimize_projections.rs @@ -15,13 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! Optimizer rule to prune unnecessary columns from intermediate schemas -//! inside the [`LogicalPlan`]. This rule: -//! - Removes unnecessary columns that do not appear at the output and/or are -//! not used during any computation step. -//! - Adds projections to decrease table column size before operators that -//! benefit from a smaller memory footprint at its input. -//! - Removes unnecessary [`LogicalPlan::Projection`]s from the [`LogicalPlan`]. +//! [`OptimizeProjections`] identifies and eliminates unused columns use std::collections::HashSet; use std::sync::Arc; @@ -44,7 +38,13 @@ use datafusion_expr::utils::inspect_expr_pre; use hashbrown::HashMap; use itertools::{izip, Itertools}; -/// A rule for optimizing logical plans by removing unused columns/fields. +/// Optimizer rule to prune unnecessary columns from intermediate schemas +/// inside the [`LogicalPlan`]. This rule: +/// - Removes unnecessary columns that do not appear at the output and/or are +/// not used during any computation step. +/// - Adds projections to decrease table column size before operators that +/// benefit from a smaller memory footprint at its input. +/// - Removes unnecessary [`LogicalPlan::Projection`]s from the [`LogicalPlan`]. /// /// `OptimizeProjections` is an optimizer rule that identifies and eliminates /// columns from a logical plan that are not used by downstream operations. diff --git a/datafusion/optimizer/src/optimizer.rs b/datafusion/optimizer/src/optimizer.rs index 3153f72d7ee70..03ff402c3e3f2 100644 --- a/datafusion/optimizer/src/optimizer.rs +++ b/datafusion/optimizer/src/optimizer.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! Query optimizer traits +//! [`Optimizer`] and [`OptimizerRule`] use std::collections::HashSet; use std::sync::Arc; @@ -54,7 +54,7 @@ use datafusion_expr::logical_plan::LogicalPlan; use chrono::{DateTime, Utc}; use log::{debug, warn}; -/// `OptimizerRule` transforms one [`LogicalPlan`] into another which +/// `OptimizerRule`s transforms one [`LogicalPlan`] into another which /// computes the same results, but in a potentially more efficient /// way. If there are no suitable transformations for the input plan, /// the optimizer should simply return it unmodified. diff --git a/datafusion/optimizer/src/propagate_empty_relation.rs b/datafusion/optimizer/src/propagate_empty_relation.rs index 55fb982d2a875..2aca6f93254ad 100644 --- a/datafusion/optimizer/src/propagate_empty_relation.rs +++ b/datafusion/optimizer/src/propagate_empty_relation.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +//! [`PropagateEmptyRelation`] eliminates nodes fed by `EmptyRelation` use datafusion_common::{plan_err, Result}; use datafusion_expr::logical_plan::LogicalPlan; use datafusion_expr::{EmptyRelation, JoinType, Projection, Union}; diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index 83db4b0640a49..ff24df259adfd 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -12,8 +12,7 @@ // specific language governing permissions and limitations // under the License. -//! [`PushDownFilter`] Moves filters so they are applied as early as possible in -//! the plan. +//! [`PushDownFilter`] applies filters as early as possible use std::collections::{HashMap, HashSet}; use std::sync::Arc; diff --git a/datafusion/optimizer/src/push_down_limit.rs b/datafusion/optimizer/src/push_down_limit.rs index 33d02d5c5628e..cca6c3fd9bd17 100644 --- a/datafusion/optimizer/src/push_down_limit.rs +++ b/datafusion/optimizer/src/push_down_limit.rs @@ -15,8 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! Optimizer rule to push down LIMIT in the query plan -//! It will push down through projection, limits (taking the smaller limit) +//! [`PushDownLimit`] pushes `LIMIT` earlier in the query plan use std::sync::Arc; @@ -29,7 +28,9 @@ use datafusion_expr::logical_plan::{ }; use datafusion_expr::CrossJoin; -/// Optimization rule that tries to push down LIMIT. +/// Optimization rule that tries to push down `LIMIT`. +/// +//. It will push down through projection, limits (taking the smaller limit) #[derive(Default)] pub struct PushDownLimit {} diff --git a/datafusion/optimizer/src/push_down_projection.rs b/datafusion/optimizer/src/push_down_projection.rs index ccdcf2f65bc8f..ae57ed9e5a345 100644 --- a/datafusion/optimizer/src/push_down_projection.rs +++ b/datafusion/optimizer/src/push_down_projection.rs @@ -15,9 +15,6 @@ // specific language governing permissions and limitations // under the License. -//! Projection Push Down optimizer rule ensures that only referenced columns are -//! loaded into memory - #[cfg(test)] mod tests { use std::collections::HashMap; diff --git a/datafusion/optimizer/src/replace_distinct_aggregate.rs b/datafusion/optimizer/src/replace_distinct_aggregate.rs index 0055e329c29d9..752915be69c04 100644 --- a/datafusion/optimizer/src/replace_distinct_aggregate.rs +++ b/datafusion/optimizer/src/replace_distinct_aggregate.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +//! [`ReplaceDistinctWithAggregate`] replaces `DISTINCT ...` with `GROUP BY ...` use crate::optimizer::{ApplyOrder, ApplyOrder::BottomUp}; use crate::{OptimizerConfig, OptimizerRule}; diff --git a/datafusion/optimizer/src/rewrite_disjunctive_predicate.rs b/datafusion/optimizer/src/rewrite_disjunctive_predicate.rs index 90c96b4b8b8cb..059b1452ff3dc 100644 --- a/datafusion/optimizer/src/rewrite_disjunctive_predicate.rs +++ b/datafusion/optimizer/src/rewrite_disjunctive_predicate.rs @@ -15,6 +15,8 @@ // specific language governing permissions and limitations // under the License. +//! [`RewriteDisjunctivePredicate`] rewrites predicates to reduce redundancy + use crate::optimizer::ApplyOrder; use crate::{OptimizerConfig, OptimizerRule}; use datafusion_common::Result; diff --git a/datafusion/optimizer/src/scalar_subquery_to_join.rs b/datafusion/optimizer/src/scalar_subquery_to_join.rs index 8acc36e479cab..a2c4eabcaae6e 100644 --- a/datafusion/optimizer/src/scalar_subquery_to_join.rs +++ b/datafusion/optimizer/src/scalar_subquery_to_join.rs @@ -15,6 +15,8 @@ // specific language governing permissions and limitations // under the License. +//! [`ScalarSubqueryToJoin`] rewriting scalar subquery filters to `JOIN`s + use std::collections::{BTreeSet, HashMap}; use std::sync::Arc; diff --git a/datafusion/optimizer/src/simplify_expressions/mod.rs b/datafusion/optimizer/src/simplify_expressions/mod.rs index 5244f9a5af881..d0399fef07e64 100644 --- a/datafusion/optimizer/src/simplify_expressions/mod.rs +++ b/datafusion/optimizer/src/simplify_expressions/mod.rs @@ -15,6 +15,9 @@ // specific language governing permissions and limitations // under the License. +//! [`SimplifyExpressions`] simplifies expressions in the logical plan, +//! [`ExprSimplifier`] simplifies individual `Expr`s. + pub mod expr_simplifier; mod guarantees; mod inlist_simplifier; diff --git a/datafusion/optimizer/src/single_distinct_to_groupby.rs b/datafusion/optimizer/src/single_distinct_to_groupby.rs index 5b47abb308d0d..076bf4e24296d 100644 --- a/datafusion/optimizer/src/single_distinct_to_groupby.rs +++ b/datafusion/optimizer/src/single_distinct_to_groupby.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! single distinct to group by optimizer rule +//! [`SingleDistinctToGroupBy`] replaces `AGG(DISTINCT ..)` with `AGG(..) GROUP BY ..` use std::sync::Arc; diff --git a/datafusion/optimizer/src/unwrap_cast_in_comparison.rs b/datafusion/optimizer/src/unwrap_cast_in_comparison.rs index f573ac69377ba..fda390f379610 100644 --- a/datafusion/optimizer/src/unwrap_cast_in_comparison.rs +++ b/datafusion/optimizer/src/unwrap_cast_in_comparison.rs @@ -15,9 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! Unwrap-cast binary comparison rule can be used to the binary/inlist comparison expr now, and other type -//! of expr can be added if needed. -//! This rule can reduce adding the `Expr::Cast` the expr instead of adding the `Expr::Cast` to literal expr. +//! [`UnwrapCastInComparison`] rewrites `CAST(col) = lit` to `col = CAST(lit)` use std::cmp::Ordering; use std::sync::Arc; diff --git a/datafusion/optimizer/src/utils.rs b/datafusion/optimizer/src/utils.rs index 0df79550f143a..560c63b18882a 100644 --- a/datafusion/optimizer/src/utils.rs +++ b/datafusion/optimizer/src/utils.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! Collection of utility functions that are leveraged by the query optimizer rules +//! Utility functions leveraged by the query optimizer rules use std::collections::{BTreeSet, HashMap};