diff --git a/benchmarks/expected-plans/q19.txt b/benchmarks/expected-plans/q19.txt index ff729db9658a..a8ce0e6c8527 100644 --- a/benchmarks/expected-plans/q19.txt +++ b/benchmarks/expected-plans/q19.txt @@ -3,7 +3,7 @@ Projection: SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS re Projection: lineitem.l_extendedprice, lineitem.l_discount Filter: part.p_brand = Utf8("Brand#12") AND part.p_container IN ([Utf8("SM CASE"), Utf8("SM BOX"), Utf8("SM PACK"), Utf8("SM PKG")]) AND lineitem.l_quantity >= Decimal128(Some(100),15,2) AND lineitem.l_quantity <= Decimal128(Some(1100),15,2) AND part.p_size <= Int32(5) OR part.p_brand = Utf8("Brand#23") AND part.p_container IN ([Utf8("MED BAG"), Utf8("MED BOX"), Utf8("MED PKG"), Utf8("MED PACK")]) AND lineitem.l_quantity >= Decimal128(Some(1000),15,2) AND lineitem.l_quantity <= Decimal128(Some(2000),15,2) AND part.p_size <= Int32(10) OR part.p_brand = Utf8("Brand#34") AND part.p_container IN ([Utf8("LG CASE"), Utf8("LG BOX"), Utf8("LG PACK"), Utf8("LG PKG")]) AND lineitem.l_quantity >= Decimal128(Some(2000),15,2) AND lineitem.l_quantity <= Decimal128(Some(3000),15,2) AND part.p_size <= Int32(15) Inner Join: lineitem.l_partkey = part.p_partkey - Filter: lineitem.l_shipmode IN ([Utf8("AIR"), Utf8("AIR REG")]) AND lineitem.l_shipinstruct = Utf8("DELIVER IN PERSON") AND lineitem.l_quantity >= Decimal128(Some(100),15,2) AND lineitem.l_quantity <= Decimal128(Some(1100),15,2) OR lineitem.l_quantity >= Decimal128(Some(1000),15,2) AND lineitem.l_quantity <= Decimal128(Some(2000),15,2) OR lineitem.l_quantity >= Decimal128(Some(2000),15,2) AND lineitem.l_quantity <= Decimal128(Some(3000),15,2) + Filter: lineitem.l_quantity >= Decimal128(Some(100),15,2) AND lineitem.l_quantity <= Decimal128(Some(1100),15,2) OR lineitem.l_quantity >= Decimal128(Some(1000),15,2) AND lineitem.l_quantity <= Decimal128(Some(2000),15,2) OR lineitem.l_quantity >= Decimal128(Some(2000),15,2) AND lineitem.l_quantity <= Decimal128(Some(3000),15,2) AND lineitem.l_shipmode IN ([Utf8("AIR"), Utf8("AIR REG")]) AND lineitem.l_shipinstruct = Utf8("DELIVER IN PERSON") TableScan: lineitem projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode] - Filter: part.p_size >= Int32(1) AND part.p_brand = Utf8("Brand#12") AND part.p_container IN ([Utf8("SM CASE"), Utf8("SM BOX"), Utf8("SM PACK"), Utf8("SM PKG")]) AND part.p_size <= Int32(5) OR part.p_brand = Utf8("Brand#23") AND part.p_container IN ([Utf8("MED BAG"), Utf8("MED BOX"), Utf8("MED PKG"), Utf8("MED PACK")]) AND part.p_size <= Int32(10) OR part.p_brand = Utf8("Brand#34") AND part.p_container IN ([Utf8("LG CASE"), Utf8("LG BOX"), Utf8("LG PACK"), Utf8("LG PKG")]) AND part.p_size <= Int32(15) - TableScan: part projection=[p_partkey, p_brand, p_size, p_container] + Filter: part.p_brand = Utf8("Brand#12") AND part.p_container IN ([Utf8("SM CASE"), Utf8("SM BOX"), Utf8("SM PACK"), Utf8("SM PKG")]) AND part.p_size <= Int32(5) OR part.p_brand = Utf8("Brand#23") AND part.p_container IN ([Utf8("MED BAG"), Utf8("MED BOX"), Utf8("MED PKG"), Utf8("MED PACK")]) AND part.p_size <= Int32(10) OR part.p_brand = Utf8("Brand#34") AND part.p_container IN ([Utf8("LG CASE"), Utf8("LG BOX"), Utf8("LG PACK"), Utf8("LG PKG")]) AND part.p_size <= Int32(15) AND part.p_size >= Int32(1) + TableScan: part projection=[p_partkey, p_brand, p_size, p_container] \ No newline at end of file diff --git a/datafusion/core/src/config.rs b/datafusion/core/src/config.rs index b95c12d3ba1a..e058f5c72bd4 100644 --- a/datafusion/core/src/config.rs +++ b/datafusion/core/src/config.rs @@ -64,6 +64,9 @@ pub const OPT_PARQUET_ENABLE_PAGE_INDEX: &str = pub const OPT_OPTIMIZER_SKIP_FAILED_RULES: &str = "datafusion.optimizer.skip_failed_rules"; +/// Configuration option "datafusion.optimizer.max_passes" +pub const OPT_OPTIMIZER_MAX_PASSES: &str = "datafusion.optimizer.max_passes"; + /// Definition of a configuration option pub struct ConfigDefinition { /// key used to identifier this configuration option @@ -241,7 +244,11 @@ impl BuiltInConfigs { rule. When set to false, any rules that produce errors will cause the query to fail.", true ), - ] + ConfigDefinition::new_u64( + OPT_OPTIMIZER_MAX_PASSES, + "Number of times that the optimizer will attempt to optimize the plan", + 3 + )] } } diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context.rs index c50f79426c8f..f140ce1c3b98 100644 --- a/datafusion/core/src/execution/context.rs +++ b/datafusion/core/src/execution/context.rs @@ -78,7 +78,7 @@ use crate::physical_optimizer::repartition::Repartition; use crate::config::{ ConfigOptions, OPT_BATCH_SIZE, OPT_COALESCE_BATCHES, OPT_COALESCE_TARGET_BATCH_SIZE, - OPT_FILTER_NULL_JOIN_KEYS, OPT_OPTIMIZER_SKIP_FAILED_RULES, + OPT_FILTER_NULL_JOIN_KEYS, OPT_OPTIMIZER_MAX_PASSES, OPT_OPTIMIZER_SKIP_FAILED_RULES, }; use crate::datasource::file_format::file_type::{FileCompressionType, FileType}; use crate::execution::{runtime_env::RuntimeEnv, FunctionRegistry}; @@ -1599,6 +1599,13 @@ impl SessionState { .get_bool(OPT_OPTIMIZER_SKIP_FAILED_RULES) .unwrap_or_default(), ) + .with_max_passes( + self.config + .config_options + .read() + .get_u64(OPT_OPTIMIZER_MAX_PASSES) + .unwrap_or_default() as u8, + ) .with_query_execution_start_time( self.execution_props.query_execution_start_time, ); diff --git a/datafusion/core/tests/sql/information_schema.rs b/datafusion/core/tests/sql/information_schema.rs index 873ead46258b..ce318e6c2625 100644 --- a/datafusion/core/tests/sql/information_schema.rs +++ b/datafusion/core/tests/sql/information_schema.rs @@ -708,6 +708,7 @@ async fn show_all() { "| datafusion.explain.logical_plan_only | false |", "| datafusion.explain.physical_plan_only | false |", "| datafusion.optimizer.filter_null_join_keys | false |", + "| datafusion.optimizer.max_passes | 3 |", "| datafusion.optimizer.skip_failed_rules | true |", "+-------------------------------------------------+---------+", ]; diff --git a/datafusion/core/tests/sql/subqueries.rs b/datafusion/core/tests/sql/subqueries.rs index fd25e95bb422..803c24995cbc 100644 --- a/datafusion/core/tests/sql/subqueries.rs +++ b/datafusion/core/tests/sql/subqueries.rs @@ -394,9 +394,9 @@ order by cntrycode;"#; Projection: AVG(customer.c_acctbal) AS __value, alias=__sq_1 Aggregate: groupBy=[[]], aggr=[[AVG(customer.c_acctbal)]] Filter: customer.c_acctbal > Decimal128(Some(0),15,2) AND substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8("13"), Utf8("31"), Utf8("23"), Utf8("29"), Utf8("30"), Utf8("18"), Utf8("17")]) - TableScan: customer projection=[c_phone, c_acctbal], partial_filters=[CAST(customer.c_acctbal AS Decimal128(30, 15)) > Decimal128(Some(0),30,15), substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8("13"), Utf8("31"), Utf8("23"), Utf8("29"), Utf8("30"), Utf8("18"), Utf8("17")])]"# + TableScan: customer projection=[c_phone, c_acctbal], partial_filters=[CAST(customer.c_acctbal AS Decimal128(30, 15)) > Decimal128(Some(0),30,15), substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8("13"), Utf8("31"), Utf8("23"), Utf8("29"), Utf8("30"), Utf8("18"), Utf8("17")]), customer.c_acctbal > Decimal128(Some(0),15,2)]"# .to_string(); - assert_eq!(actual, expected); + assert_eq!(expected, actual); // assert data let results = execute_to_batches(&ctx, sql).await; diff --git a/datafusion/optimizer/src/optimizer.rs b/datafusion/optimizer/src/optimizer.rs index 5052dc67e225..976131e047a8 100644 --- a/datafusion/optimizer/src/optimizer.rs +++ b/datafusion/optimizer/src/optimizer.rs @@ -40,6 +40,7 @@ use datafusion_common::{DataFusionError, Result}; use datafusion_expr::logical_plan::LogicalPlan; use log::{debug, trace, warn}; use std::sync::Arc; +use std::time::Instant; /// `OptimizerRule` transforms one ['LogicalPlan'] into another which /// computes the same results, but in a potentially more efficient @@ -71,6 +72,8 @@ pub struct OptimizerConfig { skip_failing_rules: bool, /// Specify whether to enable the filter_null_keys rule filter_null_keys: bool, + /// Maximum number of times to run optimizer against a plan + max_passes: u8, } impl OptimizerConfig { @@ -81,6 +84,7 @@ impl OptimizerConfig { next_id: 0, // useful for generating things like unique subquery aliases skip_failing_rules: true, filter_null_keys: true, + max_passes: 3, } } @@ -107,6 +111,12 @@ impl OptimizerConfig { self } + /// Specify how many times to attempt to optimize the plan + pub fn with_max_passes(mut self, v: u8) -> Self { + self.max_passes = v; + self + } + /// Generate the next ID needed pub fn next_id(&mut self) -> usize { self.next_id += 1; @@ -189,38 +199,57 @@ impl Optimizer { where F: FnMut(&LogicalPlan, &dyn OptimizerRule), { + let start_time = Instant::now(); + let mut plan_str = format!("{}", plan.display_indent()); let mut new_plan = plan.clone(); - log_plan("Optimizer input", plan); + let mut i = 0; + while i < optimizer_config.max_passes { + log_plan(&format!("Optimizer input (pass {})", i), &new_plan); - for rule in &self.rules { - let result = rule.optimize(&new_plan, optimizer_config); - match result { - Ok(plan) => { - new_plan = plan; - observer(&new_plan, rule.as_ref()); - log_plan(rule.name(), &new_plan); - } - Err(ref e) => { - if optimizer_config.skip_failing_rules { - // Note to future readers: if you see this warning it signals a - // bug in the DataFusion optimizer. Please consider filing a ticket - // https://github.com/apache/arrow-datafusion - warn!( + for rule in &self.rules { + let result = rule.optimize(&new_plan, optimizer_config); + match result { + Ok(plan) => { + new_plan = plan; + observer(&new_plan, rule.as_ref()); + log_plan(rule.name(), &new_plan); + } + Err(ref e) => { + if optimizer_config.skip_failing_rules { + // Note to future readers: if you see this warning it signals a + // bug in the DataFusion optimizer. Please consider filing a ticket + // https://github.com/apache/arrow-datafusion + warn!( "Skipping optimizer rule '{}' due to unexpected error: {}", rule.name(), e ); - } else { - return Err(DataFusionError::Internal(format!( - "Optimizer rule '{}' failed due to unexpected error: {}", - rule.name(), - e - ))); + } else { + return Err(DataFusionError::Internal(format!( + "Optimizer rule '{}' failed due to unexpected error: {}", + rule.name(), + e + ))); + } } } } + log_plan(&format!("Optimized plan (pass {})", i), &new_plan); + + // TODO this is an expensive way to see if the optimizer did anything and + // it would be better to change the OptimizerRule trait to return an Option + // instead + let new_plan_str = format!("{}", new_plan.display_indent()); + if plan_str == new_plan_str { + // plan did not change, so no need to continue trying to optimize + debug!("optimizer pass {} did not make changes", i); + break; + } + plan_str = new_plan_str; + i += 1; } - log_plan("Optimized plan", &new_plan); + log_plan("Final optimized plan", &new_plan); + debug!("Optimizer took {} ms", start_time.elapsed().as_millis()); Ok(new_plan) } } diff --git a/datafusion/optimizer/src/projection_push_down.rs b/datafusion/optimizer/src/projection_push_down.rs index 9e61ccd6e229..f6430b87a4a7 100644 --- a/datafusion/optimizer/src/projection_push_down.rs +++ b/datafusion/optimizer/src/projection_push_down.rs @@ -135,7 +135,7 @@ fn optimize_plan( _optimizer_config: &OptimizerConfig, ) -> Result { let mut new_required_columns = required_columns.clone(); - match plan { + let new_plan = match plan { LogicalPlan::Projection(Projection { input, expr, @@ -509,7 +509,25 @@ fn optimize_plan( from_plan(plan, &expr, &new_inputs) } - } + }; + + // when this rule is applied multiple times it will insert duplicate nested projections, + // so we catch this here + let with_dupe_projection_removed = match new_plan? { + LogicalPlan::Projection(p) => match p.input.as_ref() { + LogicalPlan::Projection(p2) if projection_equal(&p, p2) => { + LogicalPlan::Projection(p2.clone()) + } + _ => LogicalPlan::Projection(p), + }, + other => other, + }; + + Ok(with_dupe_projection_removed) +} + +fn projection_equal(p: &Projection, p2: &Projection) -> bool { + p.expr.len() == p2.expr.len() && p.expr.iter().zip(&p2.expr).all(|(l, r)| l == r) } #[cfg(test)] diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 079cdacd1caa..2404131da3d0 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -40,9 +40,15 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.execution.batch_size | UInt64 | 8192 | Default batch size while creating new batches, it's especially useful for buffer-in-memory batches since creating tiny batches would results in too much metadata memory consumption. | | datafusion.execution.coalesce_batches | Boolean | true | When set to true, record batches will be examined between each operator and small batches will be coalesced into larger batches. This is helpful when there are highly selective filters or joins that could produce tiny output batches. The target batch size is determined by the configuration setting 'datafusion.execution.coalesce_target_batch_size'. | | datafusion.execution.coalesce_target_batch_size | UInt64 | 4096 | Target batch size when coalescing batches. Uses in conjunction with the configuration setting 'datafusion.execution.coalesce_batches'. | +| datafusion.execution.parquet.enable_page_index | Boolean | false | If true, uses parquet data page level metadata (Page Index) statistics to reduce the number of rows decoded. | +| datafusion.execution.parquet.pushdown_filters | Boolean | false | If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded. | +| datafusion.execution.parquet.reorder_filters | Boolean | false | If true, filter expressions evaluated during the parquet decoding opearation will be reordered heuristically to minimize the cost of evaluation. If false, the filters are applied in the same order as written in the query. | +| datafusion.execution.time_zone | Utf8 | UTC | The session time zone which some function require e.g. EXTRACT(HOUR from SOME_TIME) shift the underline datetime according to the time zone, | +| then extract the hour. | | datafusion.execution.time_zone | Utf8 | UTC | The session time zone which some function require e.g. EXTRACT(HOUR from SOME_TIME) shift the underline datetime according to the time zone, | | then extract the hour | | datafusion.explain.logical_plan_only | Boolean | false | When set to true, the explain statement will only print logical plans. | | datafusion.explain.physical_plan_only | Boolean | false | When set to true, the explain statement will only print physical plans. | | datafusion.optimizer.filter_null_join_keys | Boolean | false | When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. | +| datafusion.optimizer.max_passes | UInt64 | 3 | Number of times that the optimizer will attempt to optimize the plan | | datafusion.optimizer.skip_failed_rules | Boolean | true | When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail. |