diff --git a/datafusion-examples/examples/query_planning/optimizer_rule.rs b/datafusion-examples/examples/query_planning/optimizer_rule.rs index de9de7737a6a0..df676e37c9cd4 100644 --- a/datafusion-examples/examples/query_planning/optimizer_rule.rs +++ b/datafusion-examples/examples/query_planning/optimizer_rule.rs @@ -54,11 +54,10 @@ pub async fn optimizer_rule() -> Result<()> { let plan = ctx.sql(sql).await?.into_optimized_plan()?; // We can see the effect of our rewrite on the output plan that the filter - // has been rewritten to `my_eq` + // has been rewritten to `my_eq` (consolidated into TableScan as unsupported_filters) assert_eq!( plan.display_indent().to_string(), - "Filter: my_eq(person.age, Int32(22))\ - \n TableScan: person projection=[name, age]" + "TableScan: person projection=[name, age], unsupported_filters=[my_eq(person.age, Int32(22))]" ); // The query below doesn't respect a filter `where age = 22` because diff --git a/datafusion/core/src/dataframe/parquet.rs b/datafusion/core/src/dataframe/parquet.rs index 54dadfd78cbc2..4dde62666cea5 100644 --- a/datafusion/core/src/dataframe/parquet.rs +++ b/datafusion/core/src/dataframe/parquet.rs @@ -150,9 +150,11 @@ mod tests { .select_columns(&["bool_col", "int_col"])?; let plan = df.explain(false, false)?.collect().await?; - // Filters all the way to Parquet + // Filters all the way to Parquet - the physical planner creates FilterExec + // for inexact filters (parquet predicate pushdown is inexact). Column index + // is @2 because we expand projection to include filter columns. let formatted = pretty::pretty_format_batches(&plan)?.to_string(); - assert!(formatted.contains("FilterExec: id@0 = 1"), "{formatted}"); + assert!(formatted.contains("FilterExec: id@2 = 1"), "{formatted}"); Ok(()) } diff --git a/datafusion/core/src/datasource/view_test.rs b/datafusion/core/src/datasource/view_test.rs index 35418d6dea632..4511b109d75bb 100644 --- a/datafusion/core/src/datasource/view_test.rs +++ b/datafusion/core/src/datasource/view_test.rs @@ -322,11 +322,13 @@ mod tests { let plan = df.explain(false, false)?.collect().await?; - // Filters all the way to Parquet + // Filters all the way to Parquet - physical planner creates FilterExec + // for inexact filters. Column index is @2 because we expand projection + // to include filter columns. let formatted = arrow::util::pretty::pretty_format_batches(&plan) .unwrap() .to_string(); - assert!(formatted.contains("FilterExec: id@0 = 1")); + assert!(formatted.contains("FilterExec: id@2 = 1"), "{formatted}"); Ok(()) } @@ -396,11 +398,12 @@ mod tests { .await?; let plan = dataframe.into_optimized_plan()?; let actual = format!("{}", plan.display_indent()); + // Filters are now pushed to TableScan.filters during logical optimization + // (classification happens in physical planner) let expected = "\ Explain\ \n CreateView: Bare { table: \"xyz\" }\ - \n Filter: abc.column2 = Int64(5)\ - \n TableScan: abc projection=[column1, column2, column3]"; + \n TableScan: abc projection=[column1, column2, column3], unsupported_filters=[abc.column2 = Int64(5)]"; assert_eq!(expected, actual); let dataframe = session_ctx @@ -408,11 +411,11 @@ mod tests { .await?; let plan = dataframe.into_optimized_plan()?; let actual = format!("{}", plan.display_indent()); + // Filters are now pushed to TableScan.filters during logical optimization let expected = "\ Explain\ \n CreateView: Bare { table: \"xyz\" }\ - \n Filter: abc.column2 = Int64(5)\ - \n TableScan: abc projection=[column1, column2]"; + \n TableScan: abc projection=[column1, column2], unsupported_filters=[abc.column2 = Int64(5)]"; assert_eq!(expected, actual); Ok(()) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index b1aa850284aee..2dc3b8495d740 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -462,18 +462,133 @@ impl DefaultPhysicalPlanner { fetch, .. }) => { - let source = source_as_provider(source)?; + use datafusion_expr::TableProviderFilterPushDown; + use datafusion_expr::utils::conjunction; + + let provider = source_as_provider(source)?; + let source_schema = provider.schema(); + // Remove all qualifiers from the scan as the provider // doesn't know (nor should care) how the relation was // referred to in the query - let filters = unnormalize_cols(filters.iter().cloned()); - let filters_vec = filters.into_iter().collect::>(); + let filters: Vec = unnormalize_cols(filters.iter().cloned()); + + // Separate volatile filters (never push to source) + let (volatile_filters, non_volatile_filters): (Vec, Vec) = + filters.into_iter().partition(|pred| pred.is_volatile()); + + // Classify non-volatile filters using provider + let filter_refs: Vec<&Expr> = non_volatile_filters.iter().collect(); + let supported = provider.supports_filters_pushdown(&filter_refs)?; + + assert_eq!( + non_volatile_filters.len(), + supported.len(), + "supports_filters_pushdown returned {} results for {} filters", + supported.len(), + non_volatile_filters.len() + ); + + // Separate into pushable (Exact/Inexact) and post-scan filters + let mut pushable_filters = Vec::new(); + let mut post_scan_filters = Vec::new(); + + for (filter, support) in + non_volatile_filters.into_iter().zip(supported.iter()) + { + match support { + TableProviderFilterPushDown::Exact => { + pushable_filters.push(filter); + } + TableProviderFilterPushDown::Inexact => { + pushable_filters.push(filter.clone()); + post_scan_filters.push(filter); + } + TableProviderFilterPushDown::Unsupported => { + post_scan_filters.push(filter); + } + } + } + + // Add volatile filters to post-scan (never pushed) + post_scan_filters.extend(volatile_filters); + + // Compute scan projection that includes columns needed for post-scan filters + let user_projection_len = projection.as_ref().map(|p| p.len()); + let (scan_projection, needs_final_projection) = self + .compute_scan_projection_with_filters( + projection, + &post_scan_filters, + source_schema.as_ref(), + )?; + + // Limit can only be pushed if no post-filtering needed + let has_inexact = + supported.contains(&TableProviderFilterPushDown::Inexact); + let scan_limit = if has_inexact || !post_scan_filters.is_empty() { + None + } else { + *fetch + }; + + // Create the scan let opts = ScanArgs::default() - .with_projection(projection.as_deref()) - .with_filters(Some(&filters_vec)) - .with_limit(*fetch); - let res = source.scan_with_args(session_state, opts).await?; - Arc::clone(res.plan()) + .with_projection(scan_projection.as_deref()) + .with_filters(if pushable_filters.is_empty() { + None + } else { + Some(&pushable_filters) + }) + .with_limit(scan_limit); + let scan_result = provider.scan_with_args(session_state, opts).await?; + let mut plan: Arc = Arc::clone(scan_result.plan()); + + // Wrap with FilterExec if needed + if !post_scan_filters.is_empty() + && let Some(filter_expr) = conjunction(post_scan_filters.clone()) + { + let scan_df_schema = + DFSchema::try_from(plan.schema().as_ref().clone())?; + let num_scan_columns = plan.schema().fields().len(); + plan = self.create_filter_exec( + &filter_expr, + plan, + &scan_df_schema, + session_state, + num_scan_columns, + )?; + } + + // If we expanded the projection for filter columns, project back to + // only the columns the user requested + if needs_final_projection && let Some(orig_len) = user_projection_len { + let schema = plan.schema(); + let proj_exprs: Vec = (0..orig_len) + .map(|i| { + let field = schema.field(i); + let col_expr: Arc = Arc::new( + datafusion_physical_expr::expressions::Column::new( + field.name(), + i, + ), + ); + ProjectionExpr { + expr: col_expr, + alias: field.name().to_string(), + } + }) + .collect(); + plan = Arc::new(ProjectionExec::try_new(proj_exprs, plan)?); + } + + // Apply limit if not pushed to scan + if let Some(limit) = *fetch + && scan_limit.is_none() + { + plan = Arc::new(GlobalLimitExec::new(plan, 0, Some(limit))); + } + + plan } LogicalPlan::Values(Values { values, schema }) => { let exprs = values @@ -1959,7 +2074,7 @@ fn get_physical_expr_pair( } /// Extract filter predicates from a DML input plan (DELETE/UPDATE). -/// Walks the logical plan tree and collects Filter predicates, +/// Walks the logical plan tree and collects Filter predicates and TableScan filters, /// splitting AND conjunctions into individual expressions. /// Column qualifiers are stripped so expressions can be evaluated against /// the TableProvider's schema. @@ -1968,9 +2083,17 @@ fn extract_dml_filters(input: &Arc) -> Result> { let mut filters = Vec::new(); input.apply(|node| { - if let LogicalPlan::Filter(filter) = node { - // Split AND predicates into individual expressions - filters.extend(split_conjunction(&filter.predicate).into_iter().cloned()); + match node { + LogicalPlan::Filter(filter) => { + // Split AND predicates into individual expressions + filters.extend(split_conjunction(&filter.predicate).into_iter().cloned()); + } + LogicalPlan::TableScan(scan) => { + // Also extract filters from TableScan (now that all filters are + // pushed to TableScan.filters during logical optimization) + filters.extend(scan.filters.iter().cloned()); + } + _ => {} } Ok(TreeNodeRecursion::Continue) })?; @@ -2679,6 +2802,109 @@ impl DefaultPhysicalPlanner { } } + /// Compute column indices needed for the scan, including columns from + /// projection AND post-scan filters. + /// + /// Returns (scan_projection, needs_final_projection) where: + /// - scan_projection: The projection to pass to the scan + /// - needs_final_projection: true if we added extra columns for filtering + /// that need to be projected away afterwards + fn compute_scan_projection_with_filters( + &self, + projection: &Option>, + post_filters: &[Expr], + source_schema: &Schema, + ) -> Result<(Option>, bool)> { + use datafusion_expr::utils::expr_to_columns; + use std::collections::HashSet; + + // If no projection, we use all columns anyway + if projection.is_none() { + return Ok((None, false)); + } + + // If no post-filters, just use the original projection + if post_filters.is_empty() { + return Ok((projection.clone(), false)); + } + + // Collect columns referenced by post-scan filters + let mut filter_columns = HashSet::new(); + for filter in post_filters { + expr_to_columns(filter, &mut filter_columns)?; + } + + // Get filter column indices in the source schema + let filter_indices: Vec = filter_columns + .iter() + .filter_map(|col| source_schema.index_of(col.name()).ok()) + .collect(); + + let proj_indices = projection.as_ref().unwrap(); + + // Check if all filter columns are already in the projection + let proj_set: HashSet = proj_indices.iter().copied().collect(); + let missing_indices: Vec = filter_indices + .iter() + .filter(|idx| !proj_set.contains(idx)) + .copied() + .collect(); + + if missing_indices.is_empty() { + // All filter columns are in the projection, no expansion needed + return Ok((projection.clone(), false)); + } + + // Build expanded projection: original projection + missing filter columns + // We keep the original projection columns first (in their original order) + // then append the missing filter columns + let mut expanded: Vec = proj_indices.clone(); + expanded.extend(missing_indices); + + Ok((Some(expanded), true)) + } + + /// Create a FilterExec that handles async UDFs properly. + fn create_filter_exec( + &self, + predicate: &Expr, + input: Arc, + input_dfschema: &DFSchema, + session_state: &SessionState, + num_input_columns: usize, + ) -> Result> { + let runtime_expr = + self.create_physical_expr(predicate, input_dfschema, session_state)?; + let input_schema = input.schema(); + + let filter = match self.try_plan_async_exprs( + num_input_columns, + PlannedExprResult::Expr(vec![runtime_expr]), + input_schema.as_ref(), + )? { + PlanAsyncExpr::Sync(PlannedExprResult::Expr(exprs)) => { + FilterExecBuilder::new(Arc::clone(&exprs[0]), input) + .with_batch_size(session_state.config().batch_size()) + .build()? + } + PlanAsyncExpr::Async(async_map, PlannedExprResult::Expr(exprs)) => { + let async_exec = AsyncFuncExec::try_new(async_map.async_exprs, input)?; + FilterExecBuilder::new(Arc::clone(&exprs[0]), Arc::new(async_exec)) + .apply_projection(Some((0..num_input_columns).collect()))? + .with_batch_size(session_state.config().batch_size()) + .build()? + } + _ => return internal_err!("Unexpected result from try_plan_async_exprs"), + }; + + let selectivity = session_state + .config() + .options() + .optimizer + .default_filter_selectivity; + Ok(Arc::new(filter.with_default_selectivity(selectivity)?)) + } + fn try_plan_async_exprs( &self, num_input_columns: usize, diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index bab00ced1cb13..2793b5dd1b61a 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -1563,7 +1563,11 @@ async fn join_on_filter_datatype() -> Result<()> { JoinType::Inner, Some(Expr::Literal(ScalarValue::Null, None)), )?; - assert_snapshot!(join.into_optimized_plan().unwrap(), @"EmptyRelation: rows=0"); + assert_snapshot!(join.into_optimized_plan().unwrap(), @r" + Cross Join: + TableScan: a projection=[c1], partial_filters=[Boolean(NULL)] + TableScan: b projection=[c1] + "); // JOIN ON expression must be boolean type let join = left.join_on(right, JoinType::Inner, Some(lit("TRUE")))?; @@ -3137,18 +3141,18 @@ async fn test_count_wildcard_on_where_exist() -> Result<()> { assert_snapshot!( pretty_format_batches(&sql_results).unwrap(), @r" - +---------------+-----------------------------------------------------+ - | plan_type | plan | - +---------------+-----------------------------------------------------+ - | logical_plan | LeftSemi Join: | - | | TableScan: t1 projection=[a, b] | - | | SubqueryAlias: __correlated_sq_1 | - | | EmptyRelation: rows=1 | - | physical_plan | NestedLoopJoinExec: join_type=RightSemi | - | | PlaceholderRowExec | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | | - +---------------+-----------------------------------------------------+ + +---------------+------------------------------------------------------------------------+ + | plan_type | plan | + +---------------+------------------------------------------------------------------------+ + | logical_plan | LeftSemi Join: | + | | TableScan: t1 projection=[a, b], unsupported_filters=[Boolean(true)] | + | | SubqueryAlias: __correlated_sq_1 | + | | EmptyRelation: rows=1 | + | physical_plan | NestedLoopJoinExec: join_type=RightSemi | + | | PlaceholderRowExec | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | | + +---------------+------------------------------------------------------------------------+ " ); @@ -3173,18 +3177,18 @@ async fn test_count_wildcard_on_where_exist() -> Result<()> { assert_snapshot!( pretty_format_batches(&df_results).unwrap(), @r" - +---------------+-----------------------------------------------------+ - | plan_type | plan | - +---------------+-----------------------------------------------------+ - | logical_plan | LeftSemi Join: | - | | TableScan: t1 projection=[a, b] | - | | SubqueryAlias: __correlated_sq_1 | - | | EmptyRelation: rows=1 | - | physical_plan | NestedLoopJoinExec: join_type=RightSemi | - | | PlaceholderRowExec | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | | - +---------------+-----------------------------------------------------+ + +---------------+------------------------------------------------------------------------+ + | plan_type | plan | + +---------------+------------------------------------------------------------------------+ + | logical_plan | LeftSemi Join: | + | | TableScan: t1 projection=[a, b], unsupported_filters=[Boolean(true)] | + | | SubqueryAlias: __correlated_sq_1 | + | | EmptyRelation: rows=1 | + | physical_plan | NestedLoopJoinExec: join_type=RightSemi | + | | PlaceholderRowExec | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | | + +---------------+------------------------------------------------------------------------+ " ); @@ -4070,11 +4074,8 @@ async fn right_semi_with_alias_filter() -> Result<()> { actual, @r" RightSemi Join: t1.a = t2.a [a:UInt32, b:Utf8, c:Int32] - Projection: t1.a [a:UInt32] - Filter: t1.c > Int32(1) [a:UInt32, c:Int32] - TableScan: t1 projection=[a, c] [a:UInt32, c:Int32] - Filter: t2.c > Int32(1) [a:UInt32, b:Utf8, c:Int32] - TableScan: t2 projection=[a, b, c] [a:UInt32, b:Utf8, c:Int32] + TableScan: t1 projection=[a], unsupported_filters=[t1.c > Int32(1)] [a:UInt32] + TableScan: t2 projection=[a, b, c], unsupported_filters=[t2.c > Int32(1)] [a:UInt32, b:Utf8, c:Int32] " ); @@ -4117,9 +4118,7 @@ async fn right_anti_filter_push_down() -> Result<()> { actual, @r" RightAnti Join: t1.a = t2.a Filter: t2.c > Int32(1) [a:UInt32, b:Utf8, c:Int32] - Projection: t1.a [a:UInt32] - Filter: t1.c > Int32(1) [a:UInt32, c:Int32] - TableScan: t1 projection=[a, c] [a:UInt32, c:Int32] + TableScan: t1 projection=[a], unsupported_filters=[t1.c > Int32(1)] [a:UInt32] TableScan: t2 projection=[a, b, c] [a:UInt32, b:Utf8, c:Int32] " ); diff --git a/datafusion/core/tests/optimizer/mod.rs b/datafusion/core/tests/optimizer/mod.rs index 6466e9ad96d17..2227be47a9b97 100644 --- a/datafusion/core/tests/optimizer/mod.rs +++ b/datafusion/core/tests/optimizer/mod.rs @@ -77,11 +77,7 @@ fn timestamp_nano_ts_none_predicates() -> Result<()> { let plan = test_sql(sql).unwrap(); assert_snapshot!( plan, - @r" - Projection: test.col_int32 - Filter: test.col_ts_nano_none < TimestampNanosecond(1666612093000000000, None) - TableScan: test projection=[col_int32, col_ts_nano_none] - " + @"TableScan: test projection=[col_int32], unsupported_filters=[test.col_ts_nano_none < TimestampNanosecond(1666612093000000000, None)]" ); Ok(()) } @@ -97,11 +93,7 @@ fn timestamp_nano_ts_utc_predicates() { let plan = test_sql(sql).unwrap(); assert_snapshot!( plan, - @r#" - Projection: test.col_int32 - Filter: test.col_ts_nano_utc < TimestampNanosecond(1666612093000000000, Some("+00:00")) - TableScan: test projection=[col_int32, col_ts_nano_utc] - "# + @r#"TableScan: test projection=[col_int32], unsupported_filters=[test.col_ts_nano_utc < TimestampNanosecond(1666612093000000000, Some("+00:00"))]"# ); } diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 5f62f7204eff1..9b717ebd63e26 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -405,9 +405,7 @@ async fn csv_explain_plans() { actual, @r" Explain [plan_type:Utf8, plan:Utf8] - Projection: aggregate_test_100.c1 [c1:Utf8View] - Filter: aggregate_test_100.c2 > Int8(10) [c1:Utf8View, c2:Int8] - TableScan: aggregate_test_100 projection=[c1, c2], partial_filters=[aggregate_test_100.c2 > Int8(10)] [c1:Utf8View, c2:Int8] + TableScan: aggregate_test_100 projection=[c1], partial_filters=[aggregate_test_100.c2 > Int8(10)] [c1:Utf8View] " ); // @@ -418,9 +416,7 @@ async fn csv_explain_plans() { actual, @r" Explain - Projection: aggregate_test_100.c1 - Filter: aggregate_test_100.c2 > Int8(10) - TableScan: aggregate_test_100 projection=[c1, c2], partial_filters=[aggregate_test_100.c2 > Int8(10)] + TableScan: aggregate_test_100 projection=[c1], partial_filters=[aggregate_test_100.c2 > Int8(10)] " ); // @@ -438,23 +434,15 @@ async fn csv_explain_plans() { { graph[label="LogicalPlan"] 2[shape=box label="Explain"] - 3[shape=box label="Projection: aggregate_test_100.c1"] + 3[shape=box label="TableScan: aggregate_test_100 projection=[c1], partial_filters=[aggregate_test_100.c2 > Int8(10)]"] 2 -> 3 [arrowhead=none, arrowtail=normal, dir=back] - 4[shape=box label="Filter: aggregate_test_100.c2 > Int8(10)"] - 3 -> 4 [arrowhead=none, arrowtail=normal, dir=back] - 5[shape=box label="TableScan: aggregate_test_100 projection=[c1, c2], partial_filters=[aggregate_test_100.c2 > Int8(10)]"] - 4 -> 5 [arrowhead=none, arrowtail=normal, dir=back] } - subgraph cluster_6 + subgraph cluster_4 { graph[label="Detailed LogicalPlan"] - 7[shape=box label="Explain\nSchema: [plan_type:Utf8, plan:Utf8]"] - 8[shape=box label="Projection: aggregate_test_100.c1\nSchema: [c1:Utf8View]"] - 7 -> 8 [arrowhead=none, arrowtail=normal, dir=back] - 9[shape=box label="Filter: aggregate_test_100.c2 > Int8(10)\nSchema: [c1:Utf8View, c2:Int8]"] - 8 -> 9 [arrowhead=none, arrowtail=normal, dir=back] - 10[shape=box label="TableScan: aggregate_test_100 projection=[c1, c2], partial_filters=[aggregate_test_100.c2 > Int8(10)]\nSchema: [c1:Utf8View, c2:Int8]"] - 9 -> 10 [arrowhead=none, arrowtail=normal, dir=back] + 5[shape=box label="Explain\nSchema: [plan_type:Utf8, plan:Utf8]"] + 6[shape=box label="TableScan: aggregate_test_100 projection=[c1], partial_filters=[aggregate_test_100.c2 > Int8(10)]\nSchema: [c1:Utf8View]"] + 5 -> 6 [arrowhead=none, arrowtail=normal, dir=back] } } // End DataFusion GraphViz Plan @@ -474,8 +462,9 @@ async fn csv_explain_plans() { let actual = actual.into_iter().map(|r| r.join("\t")).collect::(); // Since the plan contains path that are environmentally dependant (e.g. full path of the test file), only verify important content assert_contains!(&actual, "logical_plan"); - assert_contains!(&actual, "Projection: aggregate_test_100.c1"); - assert_contains!(actual, "Filter: aggregate_test_100.c2 > Int8(10)"); + // Filters are now pushed to TableScan.filters (classification happens in physical planner) + assert_contains!(&actual, "TableScan: aggregate_test_100 projection=[c1]"); + assert_contains!(actual, "partial_filters=[aggregate_test_100.c2 > Int8(10)]"); } #[tokio::test] @@ -621,9 +610,7 @@ async fn csv_explain_verbose_plans() { actual, @r" Explain [plan_type:Utf8, plan:Utf8] - Projection: aggregate_test_100.c1 [c1:Utf8View] - Filter: aggregate_test_100.c2 > Int8(10) [c1:Utf8View, c2:Int8] - TableScan: aggregate_test_100 projection=[c1, c2], partial_filters=[aggregate_test_100.c2 > Int8(10)] [c1:Utf8View, c2:Int8] + TableScan: aggregate_test_100 projection=[c1], partial_filters=[aggregate_test_100.c2 > Int8(10)] [c1:Utf8View] " ); // @@ -634,9 +621,7 @@ async fn csv_explain_verbose_plans() { actual, @r" Explain - Projection: aggregate_test_100.c1 - Filter: aggregate_test_100.c2 > Int8(10) - TableScan: aggregate_test_100 projection=[c1, c2], partial_filters=[aggregate_test_100.c2 > Int8(10)] + TableScan: aggregate_test_100 projection=[c1], partial_filters=[aggregate_test_100.c2 > Int8(10)] " ); // @@ -654,23 +639,15 @@ async fn csv_explain_verbose_plans() { { graph[label="LogicalPlan"] 2[shape=box label="Explain"] - 3[shape=box label="Projection: aggregate_test_100.c1"] + 3[shape=box label="TableScan: aggregate_test_100 projection=[c1], partial_filters=[aggregate_test_100.c2 > Int8(10)]"] 2 -> 3 [arrowhead=none, arrowtail=normal, dir=back] - 4[shape=box label="Filter: aggregate_test_100.c2 > Int8(10)"] - 3 -> 4 [arrowhead=none, arrowtail=normal, dir=back] - 5[shape=box label="TableScan: aggregate_test_100 projection=[c1, c2], partial_filters=[aggregate_test_100.c2 > Int8(10)]"] - 4 -> 5 [arrowhead=none, arrowtail=normal, dir=back] } - subgraph cluster_6 + subgraph cluster_4 { graph[label="Detailed LogicalPlan"] - 7[shape=box label="Explain\nSchema: [plan_type:Utf8, plan:Utf8]"] - 8[shape=box label="Projection: aggregate_test_100.c1\nSchema: [c1:Utf8View]"] - 7 -> 8 [arrowhead=none, arrowtail=normal, dir=back] - 9[shape=box label="Filter: aggregate_test_100.c2 > Int8(10)\nSchema: [c1:Utf8View, c2:Int8]"] - 8 -> 9 [arrowhead=none, arrowtail=normal, dir=back] - 10[shape=box label="TableScan: aggregate_test_100 projection=[c1, c2], partial_filters=[aggregate_test_100.c2 > Int8(10)]\nSchema: [c1:Utf8View, c2:Int8]"] - 9 -> 10 [arrowhead=none, arrowtail=normal, dir=back] + 5[shape=box label="Explain\nSchema: [plan_type:Utf8, plan:Utf8]"] + 6[shape=box label="TableScan: aggregate_test_100 projection=[c1], partial_filters=[aggregate_test_100.c2 > Int8(10)]\nSchema: [c1:Utf8View]"] + 5 -> 6 [arrowhead=none, arrowtail=normal, dir=back] } } // End DataFusion GraphViz Plan diff --git a/datafusion/core/tests/sql/joins.rs b/datafusion/core/tests/sql/joins.rs index 7c0e89ee96418..3df0f07065af1 100644 --- a/datafusion/core/tests/sql/joins.rs +++ b/datafusion/core/tests/sql/joins.rs @@ -295,7 +295,7 @@ async fn unparse_cross_join() -> Result<()> { let optimized_plan = df.into_optimized_plan()?; let opt_sql = plan_to_sql(&optimized_plan)?; - assert_snapshot!(opt_sql, @"SELECT j1.j1_id, j2.j2_string FROM j1 CROSS JOIN j2 WHERE (j2.j2_id = 0)"); + assert_snapshot!(opt_sql, @"SELECT j1.j1_id, j2.j2_string FROM j1 INNER JOIN j2 ON (j2.j2_id = 0)"); Ok(()) } diff --git a/datafusion/optimizer/src/optimize_projections/mod.rs b/datafusion/optimizer/src/optimize_projections/mod.rs index f97b05ea68fbd..3572b576603fc 100644 --- a/datafusion/optimizer/src/optimize_projections/mod.rs +++ b/datafusion/optimizer/src/optimize_projections/mod.rs @@ -264,12 +264,42 @@ fn optimize_projections( // Get indices referred to in the original (schema with all fields) // given projected indices. - let projection = match &projection { + // Note: `indices` refers to the projected schema, so we map through + // the projection to get source schema indices. + let new_projection = match &projection { Some(projection) => indices.into_mapped_indices(|idx| projection[idx]), None => indices.into_inner(), }; - let new_scan = - TableScan::try_new(table_name, source, Some(projection), filters, fetch)?; + + // Include columns from filters in the projection. + // This ensures filter columns are included even when there is no + // separate Filter node above the TableScan. + // Filter columns reference the source schema directly. + // We add any filter columns that are not already in the projection, + // preserving the original projection order. + let source_schema = Arc::new(DFSchema::try_from_qualified_schema( + table_name.clone(), + &source.schema(), + )?); + let filter_indices = + RequiredIndices::new().with_exprs(&source_schema, filters.iter()); + + // Convert to a set for fast lookup, then add missing filter indices + let projection_set: HashSet = new_projection.iter().cloned().collect(); + let mut final_projection = new_projection; + for idx in filter_indices.into_inner() { + if !projection_set.contains(&idx) { + final_projection.push(idx); + } + } + + let new_scan = TableScan::try_new( + table_name, + source, + Some(final_projection), + filters, + fetch, + )?; return Ok(Transformed::yes(LogicalPlan::TableScan(new_scan))); } @@ -2173,7 +2203,8 @@ mod tests { plan, @r" Projection: Int32(1) AS a - TableScan: test projection=[], full_filters=[b = Int32(1)] + Projection: + TableScan: test projection=[b], full_filters=[b = Int32(1)] " ) } diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index ecd6a89f2a3e6..cbf06ddf9f50e 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -28,8 +28,8 @@ use datafusion_common::tree_node::{ Transformed, TransformedResult, TreeNode, TreeNodeRecursion, }; use datafusion_common::{ - Column, DFSchema, Result, assert_eq_or_internal_err, assert_or_internal_err, - internal_err, plan_err, qualified_name, + Column, DFSchema, Result, assert_or_internal_err, internal_err, plan_err, + qualified_name, }; use datafusion_expr::expr::WindowFunction; use datafusion_expr::expr_rewriter::replace_col; @@ -37,9 +37,7 @@ use datafusion_expr::logical_plan::{Join, JoinType, LogicalPlan, TableScan, Unio use datafusion_expr::utils::{ conjunction, expr_to_columns, split_conjunction, split_conjunction_owned, }; -use datafusion_expr::{ - BinaryExpr, Expr, Filter, Operator, Projection, TableProviderFilterPushDown, and, or, -}; +use datafusion_expr::{BinaryExpr, Expr, Filter, Operator, Projection, and, or}; use crate::optimizer::ApplyOrder; use crate::simplify_expressions::simplify_predicates; @@ -1128,60 +1126,38 @@ impl OptimizerRule for PushDownFilter { LogicalPlan::TableScan(scan) => { let filter_predicates = split_conjunction(&filter.predicate); - let (volatile_filters, non_volatile_filters): (Vec<&Expr>, Vec<&Expr>) = - filter_predicates - .into_iter() - .partition(|pred| pred.is_volatile()); - - // Check which non-volatile filters are supported by source - let supported_filters = scan - .source - .supports_filters_pushdown(non_volatile_filters.as_slice())?; - assert_eq_or_internal_err!( - non_volatile_filters.len(), - supported_filters.len(), - "Vec returned length: {} from supports_filters_pushdown is not the same size as the filters passed, which length is: {}", - supported_filters.len(), - non_volatile_filters.len() - ); - - // Compose scan filters from non-volatile filters of `Exact` or `Inexact` pushdown type - let zip = non_volatile_filters.into_iter().zip(supported_filters); - - let new_scan_filters = zip - .clone() - .filter(|(_, res)| res != &TableProviderFilterPushDown::Unsupported) - .map(|(pred, _)| pred); - - // Add new scan filters + // Partition: scalar subqueries must stay in Filter nodes + // (they're essentially a fork in the logical plan tree) + let (scalar_subquery_filters, pushable_filters): (Vec<_>, Vec<_>) = + filter_predicates.iter().partition(|pred| { + pred.exists(|e| Ok(matches!(e, Expr::ScalarSubquery(_)))) + .unwrap() + }); + + // Combine existing scan filters with all pushable predicates + // Filter classification (Exact/Inexact/Unsupported) is deferred to + // the physical planner let new_scan_filters: Vec = scan .filters .iter() - .chain(new_scan_filters) + .chain(pushable_filters) .unique() .cloned() .collect(); - // Compose predicates to be of `Unsupported` or `Inexact` pushdown type, and also include volatile filters - let new_predicate: Vec = zip - .filter(|(_, res)| res != &TableProviderFilterPushDown::Exact) - .map(|(pred, _)| pred) - .chain(volatile_filters) - .cloned() - .collect(); - let new_scan = LogicalPlan::TableScan(TableScan { filters: new_scan_filters, ..scan }); - Transformed::yes(new_scan).transform_data(|new_scan| { - if let Some(predicate) = conjunction(new_predicate) { - make_filter(predicate, Arc::new(new_scan)).map(Transformed::yes) - } else { - Ok(Transformed::no(new_scan)) - } - }) + // Keep only scalar subquery filters in a Filter node + let remaining: Vec = + scalar_subquery_filters.into_iter().cloned().collect(); + if let Some(predicate) = conjunction(remaining) { + make_filter(predicate, Arc::new(new_scan)).map(Transformed::yes) + } else { + Ok(Transformed::yes(new_scan)) + } } LogicalPlan::Extension(extension_plan) => { // This check prevents the Filter from being removed when the extension node has no children, @@ -1437,9 +1413,9 @@ mod tests { use datafusion_expr::logical_plan::table_scan; use datafusion_expr::{ ColumnarValue, ExprFunctionExt, Extension, LogicalPlanBuilder, - ScalarFunctionArgs, ScalarUDF, ScalarUDFImpl, Signature, TableSource, TableType, - UserDefinedLogicalNodeCore, Volatility, WindowFunctionDefinition, col, in_list, - in_subquery, lit, + ScalarFunctionArgs, ScalarUDF, ScalarUDFImpl, Signature, + TableProviderFilterPushDown, TableSource, TableType, UserDefinedLogicalNodeCore, + Volatility, WindowFunctionDefinition, col, in_list, in_subquery, lit, }; use crate::OptimizerContext; @@ -3148,10 +3124,7 @@ mod tests { assert_optimized_plan_equal!( plan, - @r" - Filter: a = Int64(1) - TableScan: test, partial_filters=[a = Int64(1)] - " + @"TableScan: test, partial_filters=[a = Int64(1)]" ) } @@ -3169,10 +3142,7 @@ mod tests { // each time. assert_optimized_plan_equal!( optimized_plan, - @r" - Filter: a = Int64(1) - TableScan: test, partial_filters=[a = Int64(1)] - " + @"TableScan: test, partial_filters=[a = Int64(1)]" ) } @@ -3183,10 +3153,7 @@ mod tests { assert_optimized_plan_equal!( plan, - @r" - Filter: a = Int64(1) - TableScan: test - " + @"TableScan: test, unsupported_filters=[a = Int64(1)]" ) } @@ -3205,8 +3172,7 @@ mod tests { plan, @r" Projection: a, b - Filter: a = Int64(10) AND b > Int64(11) - TableScan: test projection=[a], partial_filters=[a = Int64(10), b > Int64(11)] + TableScan: test projection=[a], partial_filters=[a = Int64(10), b > Int64(11)] " ) } @@ -4059,8 +4025,7 @@ mod tests { plan, @r" Projection: test.a, test.b - Filter: TestScalarUDF() > Float64(0.1) - TableScan: test + TableScan: test, full_filters=[TestScalarUDF() > Float64(0.1)] " ) } @@ -4093,8 +4058,7 @@ mod tests { plan, @r" Projection: test.a, test.b - Filter: TestScalarUDF() > Float64(0.1) - TableScan: test, full_filters=[t.a > Int32(5), t.b > Int32(10)] + TableScan: test, full_filters=[TestScalarUDF() > Float64(0.1), t.a > Int32(5), t.b > Int32(10)] " ) } @@ -4130,8 +4094,7 @@ mod tests { plan, @r" Projection: a, b - Filter: t.a > Int32(5) AND t.b > Int32(10) AND TestScalarUDF() > Float64(0.1) - TableScan: test + TableScan: test, unsupported_filters=[TestScalarUDF() > Float64(0.1), t.a > Int32(5), t.b > Int32(10)] " ) } diff --git a/datafusion/optimizer/tests/optimizer_integration.rs b/datafusion/optimizer/tests/optimizer_integration.rs index 36a6df54ddaf0..b1352ecb3649a 100644 --- a/datafusion/optimizer/tests/optimizer_integration.rs +++ b/datafusion/optimizer/tests/optimizer_integration.rs @@ -78,10 +78,8 @@ fn recursive_cte_with_nested_subquery() -> Result<()> { Projection: t.col_int32, numbers.level + Int64(1) Inner Join: CAST(t.col_int32 AS Int64) = CAST(numbers.id AS Int64) + Int64(1) SubqueryAlias: t - Filter: CAST(test.col_int32 AS Int64) IS NOT NULL - TableScan: test - Filter: CAST(numbers.id AS Int64) + Int64(1) IS NOT NULL - TableScan: numbers + TableScan: test, unsupported_filters=[CAST(test.col_int32 AS Int64) IS NOT NULL] + TableScan: numbers, unsupported_filters=[CAST(numbers.id AS Int64) + Int64(1) IS NOT NULL] " ); @@ -130,13 +128,10 @@ fn subquery_filter_with_cast() -> Result<()> { @r#" Projection: test.col_int32 Inner Join: Filter: CAST(test.col_int32 AS Float64) > __scalar_sq_1.avg(test.col_int32) - TableScan: test projection=[col_int32] + TableScan: test projection=[col_int32], unsupported_filters=[Boolean(true)] SubqueryAlias: __scalar_sq_1 Aggregate: groupBy=[[]], aggr=[[avg(CAST(test.col_int32 AS Float64))]] - Projection: test.col_int32 - Filter: __common_expr_4 >= Date32("2002-05-08") AND __common_expr_4 <= Date32("2002-05-13") - Projection: CAST(test.col_utf8 AS Date32) AS __common_expr_4, test.col_int32 - TableScan: test projection=[col_int32, col_utf8] + TableScan: test projection=[col_int32], unsupported_filters=[CAST(test.col_utf8 AS Date32) >= Date32("2002-05-08"), CAST(test.col_utf8 AS Date32) <= Date32("2002-05-13")] "# ); Ok(()) @@ -165,11 +160,7 @@ fn unsigned_target_type() -> Result<()> { assert_snapshot!( format!("{plan}"), - @r" - Projection: test.col_utf8 - Filter: test.col_uint32 > UInt32(0) - TableScan: test projection=[col_uint32, col_utf8] - " + @"TableScan: test projection=[col_utf8], unsupported_filters=[test.col_uint32 > UInt32(0)]" ); Ok(()) } @@ -203,12 +194,10 @@ fn semi_join_with_join_filter() -> Result<()> { @r" Projection: test.col_utf8 LeftSemi Join: test.col_int32 = __correlated_sq_1.col_int32 Filter: test.col_uint32 != __correlated_sq_1.col_uint32 - Filter: test.col_int32 IS NOT NULL - TableScan: test projection=[col_int32, col_uint32, col_utf8] + TableScan: test projection=[col_int32, col_uint32, col_utf8], unsupported_filters=[test.col_int32 IS NOT NULL] SubqueryAlias: __correlated_sq_1 SubqueryAlias: t2 - Filter: test.col_int32 IS NOT NULL - TableScan: test projection=[col_int32, col_uint32] + TableScan: test projection=[col_int32, col_uint32], unsupported_filters=[test.col_int32 IS NOT NULL] " ); Ok(()) @@ -230,8 +219,7 @@ fn anti_join_with_join_filter() -> Result<()> { TableScan: test projection=[col_int32, col_uint32, col_utf8] SubqueryAlias: __correlated_sq_1 SubqueryAlias: t2 - Filter: test.col_int32 IS NOT NULL - TableScan: test projection=[col_int32, col_uint32] + TableScan: test projection=[col_int32, col_uint32], unsupported_filters=[test.col_int32 IS NOT NULL] " ); Ok(()) @@ -247,13 +235,11 @@ fn where_exists_distinct() -> Result<()> { format!("{plan}"), @r" LeftSemi Join: test.col_int32 = __correlated_sq_1.col_int32 - Filter: test.col_int32 IS NOT NULL - TableScan: test projection=[col_int32] + TableScan: test projection=[col_int32], unsupported_filters=[test.col_int32 IS NOT NULL] SubqueryAlias: __correlated_sq_1 Aggregate: groupBy=[[t2.col_int32]], aggr=[[]] SubqueryAlias: t2 - Filter: test.col_int32 IS NOT NULL - TableScan: test projection=[col_int32] + TableScan: test projection=[col_int32], unsupported_filters=[test.col_int32 IS NOT NULL] " ); @@ -294,9 +280,7 @@ fn between_date32_plus_interval() -> Result<()> { format!("{plan}"), @r#" Aggregate: groupBy=[[]], aggr=[[count(Int64(1))]] - Projection: - Filter: test.col_date32 >= Date32("1998-03-18") AND test.col_date32 <= Date32("1998-06-16") - TableScan: test projection=[col_date32] + TableScan: test projection=[], unsupported_filters=[test.col_date32 >= Date32("1998-03-18"), test.col_date32 <= Date32("1998-06-16")] "# ); Ok(()) @@ -312,9 +296,7 @@ fn between_date64_plus_interval() -> Result<()> { format!("{plan}"), @r#" Aggregate: groupBy=[[]], aggr=[[count(Int64(1))]] - Projection: - Filter: test.col_date64 >= Date64("1998-03-18") AND test.col_date64 <= Date64("1998-06-16") - TableScan: test projection=[col_date64] + TableScan: test projection=[], unsupported_filters=[test.col_date64 >= Date64("1998-03-18"), test.col_date64 <= Date64("1998-06-16")] "# ); Ok(()) @@ -342,12 +324,10 @@ fn join_keys_in_subquery_alias() { @r" Inner Join: a.col_int32 = b.key SubqueryAlias: a - Filter: test.col_int32 IS NOT NULL - TableScan: test projection=[col_int32, col_uint32, col_utf8, col_date32, col_date64, col_ts_nano_none, col_ts_nano_utc] + TableScan: test projection=[col_int32, col_uint32, col_utf8, col_date32, col_date64, col_ts_nano_none, col_ts_nano_utc], unsupported_filters=[test.col_int32 IS NOT NULL] SubqueryAlias: b Projection: test.col_int32 AS key - Filter: test.col_int32 IS NOT NULL - TableScan: test projection=[col_int32] + TableScan: test projection=[col_int32], unsupported_filters=[test.col_int32 IS NOT NULL] " ); } @@ -362,16 +342,13 @@ fn join_keys_in_subquery_alias_1() { @r" Inner Join: a.col_int32 = b.key SubqueryAlias: a - Filter: test.col_int32 IS NOT NULL - TableScan: test projection=[col_int32, col_uint32, col_utf8, col_date32, col_date64, col_ts_nano_none, col_ts_nano_utc] + TableScan: test projection=[col_int32, col_uint32, col_utf8, col_date32, col_date64, col_ts_nano_none, col_ts_nano_utc], unsupported_filters=[test.col_int32 IS NOT NULL] SubqueryAlias: b Projection: test.col_int32 AS key Inner Join: test.col_int32 = c.col_int32 - Filter: test.col_int32 IS NOT NULL - TableScan: test projection=[col_int32] + TableScan: test projection=[col_int32], unsupported_filters=[test.col_int32 IS NOT NULL] SubqueryAlias: c - Filter: test.col_int32 IS NOT NULL - TableScan: test projection=[col_int32] + TableScan: test projection=[col_int32], unsupported_filters=[test.col_int32 IS NOT NULL] " ); } @@ -386,8 +363,7 @@ fn push_down_filter_groupby_expr_contains_alias() { @r" Projection: test.col_int32 + test.col_uint32 AS c, count(Int64(1)) AS count(*) Aggregate: groupBy=[[CAST(test.col_int32 AS Int64) + CAST(test.col_uint32 AS Int64)]], aggr=[[count(Int64(1))]] - Filter: CAST(test.col_int32 AS Int64) + CAST(test.col_uint32 AS Int64) > Int64(3) - TableScan: test projection=[col_int32, col_uint32] + TableScan: test projection=[col_int32, col_uint32], unsupported_filters=[CAST(test.col_int32 AS Int64) + CAST(test.col_uint32 AS Int64) > Int64(3)] " ); } @@ -422,10 +398,7 @@ fn eliminate_nested_filters() { assert_snapshot!( format!("{plan}"), - @r" - Filter: test.col_int32 > Int32(0) - TableScan: test projection=[col_int32] - " + @"TableScan: test projection=[col_int32], unsupported_filters=[test.col_int32 > Int32(0)]" ); } @@ -472,8 +445,7 @@ fn test_propagate_empty_relation_inner_join_and_unions() { Union TableScan: test projection=[col_int32] TableScan: test projection=[col_int32] - Filter: test.col_int32 < Int32(0) - TableScan: test projection=[col_int32] + TableScan: test projection=[col_int32], unsupported_filters=[test.col_int32 < Int32(0)] "); } @@ -511,13 +483,11 @@ fn select_correlated_predicate_subquery_with_uppercase_ident() { format!("{plan}"), @r" LeftSemi Join: test.col_int32 = __correlated_sq_1.COL_INT32 - Filter: test.col_int32 IS NOT NULL - TableScan: test projection=[col_int32, col_uint32, col_utf8, col_date32, col_date64, col_ts_nano_none, col_ts_nano_utc] + TableScan: test projection=[col_int32, col_uint32, col_utf8, col_date32, col_date64, col_ts_nano_none, col_ts_nano_utc], unsupported_filters=[test.col_int32 IS NOT NULL] SubqueryAlias: __correlated_sq_1 SubqueryAlias: T1 Projection: test.col_int32 AS COL_INT32 - Filter: test.col_int32 IS NOT NULL - TableScan: test projection=[col_int32] + TableScan: test projection=[col_int32], unsupported_filters=[test.col_int32 IS NOT NULL] " ); } @@ -544,8 +514,7 @@ fn recursive_cte_projection_pushdown() -> Result<()> { Projection: test.col_int32 AS id TableScan: test projection=[col_int32] Projection: CAST(CAST(nodes.id AS Int64) + Int64(1) AS Int32) AS id - Filter: nodes.id < Int32(3) - TableScan: nodes projection=[id] + TableScan: nodes projection=[id], unsupported_filters=[nodes.id < Int32(3)] " ); Ok(()) @@ -569,8 +538,7 @@ fn recursive_cte_with_aliased_self_reference() -> Result<()> { TableScan: test projection=[col_int32] Projection: CAST(CAST(child.id AS Int64) + Int64(1) AS Int32) AS id SubqueryAlias: child - Filter: nodes.id < Int32(3) - TableScan: nodes projection=[id] + TableScan: nodes projection=[id], unsupported_filters=[nodes.id < Int32(3)] ", ); Ok(()) @@ -595,11 +563,9 @@ fn recursive_cte_with_unused_columns() -> Result<()> { SubqueryAlias: series RecursiveQuery: is_distinct=false Projection: Int64(1) AS n - Filter: test.col_int32 = Int32(1) - TableScan: test projection=[col_int32] + TableScan: test projection=[], unsupported_filters=[test.col_int32 = Int32(1)] Projection: series.n + Int64(1) - Filter: series.n < Int64(3) - TableScan: series projection=[n] + TableScan: series projection=[n], unsupported_filters=[series.n < Int64(3)] " ); Ok(()) @@ -628,11 +594,9 @@ fn recursive_cte_projection_pushdown_baseline() -> Result<()> { SubqueryAlias: countdown RecursiveQuery: is_distinct=false Projection: test.col_int32 AS n - Filter: test.col_int32 = Int32(5) - TableScan: test projection=[col_int32] + TableScan: test projection=[col_int32], unsupported_filters=[test.col_int32 = Int32(5)] Projection: CAST(CAST(countdown.n AS Int64) - Int64(1) AS Int32) AS n - Filter: countdown.n > Int32(1) - TableScan: countdown projection=[n] + TableScan: countdown projection=[n], unsupported_filters=[countdown.n > Int32(1)] " ); Ok(()) diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 4c49fae4dceea..3e402317c07e3 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -6503,8 +6503,7 @@ logical_plan 02)--Aggregate: groupBy=[[aggregate_test_100.c3]], aggr=[[]] 03)----Projection: aggregate_test_100.c3 04)------Aggregate: groupBy=[[aggregate_test_100.c2, aggregate_test_100.c3]], aggr=[[]] -05)--------Filter: aggregate_test_100.c3 >= Int16(10) AND aggregate_test_100.c3 <= Int16(20) -06)----------TableScan: aggregate_test_100 projection=[c2, c3], partial_filters=[aggregate_test_100.c3 >= Int16(10), aggregate_test_100.c3 <= Int16(20)] +05)--------TableScan: aggregate_test_100 projection=[c2, c3], partial_filters=[aggregate_test_100.c3 >= Int16(10), aggregate_test_100.c3 <= Int16(20)] physical_plan 01)GlobalLimitExec: skip=0, fetch=4 02)--AggregateExec: mode=Final, gby=[c3@0 as c3], aggr=[], lim=[4] diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index 590f6b1a9ab9f..b15b945d7bd24 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -6506,8 +6506,7 @@ logical_plan 03)----SubqueryAlias: test 04)------SubqueryAlias: t 05)--------Projection: -06)----------Filter: substr(CAST(md5(CAST(generate_series().value AS Utf8View)) AS Utf8View), Int64(1), Int64(32)) IN ([Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278"), Utf8View("a"), Utf8View("b"), Utf8View("c")]) -07)------------TableScan: generate_series() projection=[value] +06)----------TableScan: generate_series() projection=[value], unsupported_filters=[substr(CAST(md5(CAST(generate_series().value AS Utf8View)) AS Utf8View), Int64(1), Int64(32)) IN ([Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278"), Utf8View("a"), Utf8View("b"), Utf8View("c")])] physical_plan 01)ProjectionExec: expr=[count(Int64(1))@0 as count(*)] 02)--AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] @@ -6534,8 +6533,7 @@ logical_plan 03)----SubqueryAlias: test 04)------SubqueryAlias: t 05)--------Projection: -06)----------Filter: substr(CAST(md5(CAST(generate_series().value AS Utf8View)) AS Utf8View), Int64(1), Int64(32)) IN ([Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278"), Utf8View("a"), Utf8View("b"), Utf8View("c")]) -07)------------TableScan: generate_series() projection=[value] +06)----------TableScan: generate_series() projection=[value], unsupported_filters=[substr(CAST(md5(CAST(generate_series().value AS Utf8View)) AS Utf8View), Int64(1), Int64(32)) IN ([Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278"), Utf8View("a"), Utf8View("b"), Utf8View("c")])] physical_plan 01)ProjectionExec: expr=[count(Int64(1))@0 as count(*)] 02)--AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] @@ -6562,8 +6560,7 @@ logical_plan 03)----SubqueryAlias: test 04)------SubqueryAlias: t 05)--------Projection: -06)----------Filter: substr(CAST(md5(CAST(generate_series().value AS Utf8View)) AS Utf8View), Int64(1), Int64(32)) IN ([Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278"), Utf8View("a"), Utf8View("b"), Utf8View("c")]) -07)------------TableScan: generate_series() projection=[value] +06)----------TableScan: generate_series() projection=[value], unsupported_filters=[substr(CAST(md5(CAST(generate_series().value AS Utf8View)) AS Utf8View), Int64(1), Int64(32)) IN ([Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278"), Utf8View("a"), Utf8View("b"), Utf8View("c")])] physical_plan 01)ProjectionExec: expr=[count(Int64(1))@0 as count(*)] 02)--AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] @@ -6590,8 +6587,7 @@ logical_plan 03)----SubqueryAlias: test 04)------SubqueryAlias: t 05)--------Projection: -06)----------Filter: substr(CAST(md5(CAST(generate_series().value AS Utf8View)) AS Utf8View), Int64(1), Int64(32)) IN ([Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278"), Utf8View("a"), Utf8View("b"), Utf8View("c")]) -07)------------TableScan: generate_series() projection=[value] +06)----------TableScan: generate_series() projection=[value], unsupported_filters=[substr(CAST(md5(CAST(generate_series().value AS Utf8View)) AS Utf8View), Int64(1), Int64(32)) IN ([Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278"), Utf8View("a"), Utf8View("b"), Utf8View("c")])] physical_plan 01)ProjectionExec: expr=[count(Int64(1))@0 as count(*)] 02)--AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] @@ -6618,8 +6614,7 @@ logical_plan 03)----SubqueryAlias: test 04)------SubqueryAlias: t 05)--------Projection: -06)----------Filter: substr(CAST(md5(CAST(generate_series().value AS Utf8View)) AS Utf8View), Int64(1), Int64(32)) IN ([Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278"), Utf8View("a"), Utf8View("b"), Utf8View("c")]) -07)------------TableScan: generate_series() projection=[value] +06)----------TableScan: generate_series() projection=[value], unsupported_filters=[substr(CAST(md5(CAST(generate_series().value AS Utf8View)) AS Utf8View), Int64(1), Int64(32)) IN ([Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278"), Utf8View("a"), Utf8View("b"), Utf8View("c")])] physical_plan 01)ProjectionExec: expr=[count(Int64(1))@0 as count(*)] 02)--AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] @@ -6648,8 +6643,7 @@ logical_plan 03)----SubqueryAlias: test 04)------SubqueryAlias: t 05)--------Projection: -06)----------Filter: substr(CAST(md5(CAST(generate_series().value AS Utf8View)) AS Utf8View), Int64(1), Int64(32)) IS NOT NULL OR Boolean(NULL) -07)------------TableScan: generate_series() projection=[value] +06)----------TableScan: generate_series() projection=[value], unsupported_filters=[substr(CAST(md5(CAST(generate_series().value AS Utf8View)) AS Utf8View), Int64(1), Int64(32)) IS NOT NULL OR Boolean(NULL)] physical_plan 01)ProjectionExec: expr=[count(Int64(1))@0 as count(*)] 02)--AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] diff --git a/datafusion/sqllogictest/test_files/arrow_files.slt b/datafusion/sqllogictest/test_files/arrow_files.slt index c3bc967bafb9e..f4585c59ba5cb 100644 --- a/datafusion/sqllogictest/test_files/arrow_files.slt +++ b/datafusion/sqllogictest/test_files/arrow_files.slt @@ -121,7 +121,9 @@ SELECT f0 FROM arrow_partitioned WHERE part = 123 ORDER BY f0 query TT EXPLAIN SELECT f0 FROM arrow_partitioned WHERE part = 456 ---- -logical_plan TableScan: arrow_partitioned projection=[f0], full_filters=[arrow_partitioned.part = Int32(456)] +logical_plan +01)Projection: arrow_partitioned.f0 +02)--TableScan: arrow_partitioned projection=[f0, part], full_filters=[arrow_partitioned.part = Int32(456)] physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/partitioned_table_arrow/part=456/data.arrow]]}, projection=[f0], file_type=arrow @@ -346,7 +348,9 @@ SELECT f0 FROM arrow_partitioned_stream WHERE part = 123 ORDER BY f0 query TT EXPLAIN SELECT f0 FROM arrow_partitioned_stream WHERE part = 456 ---- -logical_plan TableScan: arrow_partitioned_stream projection=[f0], full_filters=[arrow_partitioned_stream.part = Int32(456)] +logical_plan +01)Projection: arrow_partitioned_stream.f0 +02)--TableScan: arrow_partitioned_stream projection=[f0, part], full_filters=[arrow_partitioned_stream.part = Int32(456)] physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/partitioned_table_arrow_stream/part=456/data.arrow]]}, projection=[f0], file_type=arrow_stream diff --git a/datafusion/sqllogictest/test_files/async_udf.slt b/datafusion/sqllogictest/test_files/async_udf.slt index 0708b59e519a0..1d5f2c1b89e74 100644 --- a/datafusion/sqllogictest/test_files/async_udf.slt +++ b/datafusion/sqllogictest/test_files/async_udf.slt @@ -73,9 +73,7 @@ select * from data where async_abs(x) < 5; query TT explain select * from data where async_abs(x) < 5; ---- -logical_plan -01)Filter: async_abs(data.x) < Int32(5) -02)--TableScan: data projection=[x] +logical_plan TableScan: data projection=[x], unsupported_filters=[async_abs(data.x) < Int32(5)] physical_plan 01)FilterExec: __async_fn_0@1 < 5, projection=[x@0] 02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 diff --git a/datafusion/sqllogictest/test_files/cte.slt b/datafusion/sqllogictest/test_files/cte.slt index 4fd77be045c13..573916572d81f 100644 --- a/datafusion/sqllogictest/test_files/cte.slt +++ b/datafusion/sqllogictest/test_files/cte.slt @@ -154,8 +154,7 @@ logical_plan 03)----Projection: Int64(1) AS id 04)------EmptyRelation: rows=1 05)----Projection: nodes.id + Int64(1) AS id -06)------Filter: nodes.id < Int64(10) -07)--------TableScan: nodes projection=[id] +06)------TableScan: nodes projection=[id], unsupported_filters=[nodes.id < Int64(10)] physical_plan 01)RecursiveQueryExec: name=nodes, is_distinct=false 02)--ProjectionExec: expr=[1 as id] @@ -233,8 +232,7 @@ logical_plan 03)----RecursiveQuery: is_distinct=false 04)------TableScan: balance projection=[time, name, account_balance] 05)------Projection: balances.time + Int64(1) AS time, balances.name, balances.account_balance + Int64(10) AS account_balance -06)--------Filter: balances.time < Int64(10) -07)----------TableScan: balances projection=[time, name, account_balance] +06)--------TableScan: balances projection=[time, name, account_balance], unsupported_filters=[balances.time < Int64(10)] physical_plan 01)SortExec: expr=[time@0 ASC NULLS LAST, name@1 ASC NULLS LAST, account_balance@2 ASC NULLS LAST], preserve_partitioning=[false] 02)--RecursiveQueryExec: name=balances, is_distinct=false @@ -829,12 +827,11 @@ logical_plan 03)----Projection: Int64(1) AS val 04)------EmptyRelation: rows=1 05)----Projection: Int64(2) AS val -06)------Cross Join: -07)--------Filter: recursive_cte.val < Int64(2) -08)----------TableScan: recursive_cte -09)--------SubqueryAlias: sub_cte -10)----------Projection: Int64(2) AS val -11)------------EmptyRelation: rows=1 +06)------Cross Join: +07)--------TableScan: recursive_cte, unsupported_filters=[recursive_cte.val < Int64(2)] +08)--------SubqueryAlias: sub_cte +09)----------Projection: Int64(2) AS val +10)------------EmptyRelation: rows=1 physical_plan 01)RecursiveQueryExec: name=recursive_cte, is_distinct=false 02)--ProjectionExec: expr=[1 as val] @@ -1062,8 +1059,7 @@ logical_plan 03)----Projection: Int64(1) AS n 04)------EmptyRelation: rows=1 05)----Projection: numbers.n + Int64(1) -06)------Filter: numbers.n < Int64(10) -07)--------TableScan: numbers projection=[n] +06)------TableScan: numbers projection=[n], unsupported_filters=[numbers.n < Int64(10)] physical_plan 01)RecursiveQueryExec: name=numbers, is_distinct=false 02)--ProjectionExec: expr=[1 as n] @@ -1087,8 +1083,7 @@ logical_plan 03)----Projection: Int64(1) AS n 04)------EmptyRelation: rows=1 05)----Projection: numbers.n + Int64(1) -06)------Filter: numbers.n < Int64(10) -07)--------TableScan: numbers projection=[n] +06)------TableScan: numbers projection=[n], unsupported_filters=[numbers.n < Int64(10)] physical_plan 01)RecursiveQueryExec: name=numbers, is_distinct=false 02)--ProjectionExec: expr=[1 as n] diff --git a/datafusion/sqllogictest/test_files/dictionary.slt b/datafusion/sqllogictest/test_files/dictionary.slt index 511061cf82f06..cefc6f65fd95f 100644 --- a/datafusion/sqllogictest/test_files/dictionary.slt +++ b/datafusion/sqllogictest/test_files/dictionary.slt @@ -406,9 +406,7 @@ row1 1 query TT explain SELECT * from test where column2 = '1'; ---- -logical_plan -01)Filter: test.column2 = Dictionary(Int32, Utf8("1")) -02)--TableScan: test projection=[column1, column2] +logical_plan TableScan: test projection=[column1, column2], unsupported_filters=[test.column2 = Dictionary(Int32, Utf8("1"))] physical_plan 01)FilterExec: column2@1 = 1 02)--DataSourceExec: partitions=1, partition_sizes=[1] @@ -418,9 +416,7 @@ physical_plan query TT explain SELECT * from test where '1' = column2 ---- -logical_plan -01)Filter: test.column2 = Dictionary(Int32, Utf8("1")) -02)--TableScan: test projection=[column1, column2] +logical_plan TableScan: test projection=[column1, column2], unsupported_filters=[test.column2 = Dictionary(Int32, Utf8("1"))] physical_plan 01)FilterExec: column2@1 = 1 02)--DataSourceExec: partitions=1, partition_sizes=[1] @@ -435,9 +431,7 @@ row1 1 query TT explain SELECT * from test where column2 = 1; ---- -logical_plan -01)Filter: test.column2 = Dictionary(Int32, Utf8("1")) -02)--TableScan: test projection=[column1, column2] +logical_plan TableScan: test projection=[column1, column2], unsupported_filters=[test.column2 = Dictionary(Int32, Utf8("1"))] physical_plan 01)FilterExec: column2@1 = 1 02)--DataSourceExec: partitions=1, partition_sizes=[1] diff --git a/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt b/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt index b112d70f427f1..6f3640ee4dddf 100644 --- a/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt +++ b/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt @@ -279,8 +279,7 @@ EXPLAIN SELECT MAX(score) FROM agg_parquet WHERE category = 'alpha' logical_plan 01)Aggregate: groupBy=[[]], aggr=[[max(agg_parquet.score)]] 02)--Projection: agg_parquet.score -03)----Filter: agg_parquet.category = Utf8View("alpha") -04)------TableScan: agg_parquet projection=[category, score], partial_filters=[agg_parquet.category = Utf8View("alpha")] +03)----TableScan: agg_parquet projection=[score, category], partial_filters=[agg_parquet.category = Utf8View("alpha")] physical_plan 01)AggregateExec: mode=Final, gby=[], aggr=[max(agg_parquet.score)] 02)--CoalescePartitionsExec @@ -299,8 +298,7 @@ EXPLAIN SELECT MAX(score) FROM agg_parquet WHERE category = 'alpha' logical_plan 01)Aggregate: groupBy=[[]], aggr=[[max(agg_parquet.score)]] 02)--Projection: agg_parquet.score -03)----Filter: agg_parquet.category = Utf8View("alpha") -04)------TableScan: agg_parquet projection=[category, score], partial_filters=[agg_parquet.category = Utf8View("alpha")] +03)----TableScan: agg_parquet projection=[score, category], partial_filters=[agg_parquet.category = Utf8View("alpha")] physical_plan 01)AggregateExec: mode=Final, gby=[], aggr=[max(agg_parquet.score)] 02)--CoalescePartitionsExec diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 6f615ec391c9e..cf77f7966126e 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -40,8 +40,7 @@ explain SELECT c1 FROM aggregate_test_100 where c2 > 10 ---- logical_plan 01)Projection: aggregate_test_100.c1 -02)--Filter: aggregate_test_100.c2 > Int8(10) -03)----TableScan: aggregate_test_100 projection=[c1, c2], partial_filters=[aggregate_test_100.c2 > Int8(10)] +02)--TableScan: aggregate_test_100 projection=[c1, c2], partial_filters=[aggregate_test_100.c2 > Int8(10)] physical_plan 01)FilterExec: c2@1 > 10, projection=[c1@0] 02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 @@ -425,8 +424,8 @@ query TT explain select a from t1 where exists (select count(*) from t2); ---- logical_plan -01)LeftSemi Join: -02)--TableScan: t1 projection=[a] +01)LeftSemi Join: +02)--TableScan: t1 projection=[a], unsupported_filters=[Boolean(true)] 03)--SubqueryAlias: __correlated_sq_1 04)----EmptyRelation: rows=1 physical_plan diff --git a/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt b/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt index ec069212f5586..193211acc6edf 100644 --- a/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt +++ b/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt @@ -34,8 +34,7 @@ ORDER BY "date", "time"; ---- logical_plan 01)Sort: data.date ASC NULLS LAST, data.time ASC NULLS LAST -02)--Filter: data.ticker = Utf8View("A") -03)----TableScan: data projection=[date, ticker, time] +02)--TableScan: data projection=[date, ticker, time], unsupported_filters=[data.ticker = Utf8View("A")] physical_plan 01)SortPreservingMergeExec: [date@0 ASC NULLS LAST, time@2 ASC NULLS LAST] 02)--FilterExec: ticker@1 = A @@ -50,8 +49,7 @@ ORDER BY "time" ---- logical_plan 01)Sort: data.time ASC NULLS LAST -02)--Filter: data.ticker = Utf8View("A") AND CAST(data.time AS Date32) = data.date -03)----TableScan: data projection=[date, ticker, time] +02)--TableScan: data projection=[date, ticker, time], unsupported_filters=[data.ticker = Utf8View("A"), CAST(data.time AS Date32) = data.date] physical_plan 01)SortPreservingMergeExec: [time@2 ASC NULLS LAST] 02)--FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 @@ -66,8 +64,7 @@ ORDER BY "date" ---- logical_plan 01)Sort: data.date ASC NULLS LAST -02)--Filter: data.ticker = Utf8View("A") AND CAST(data.time AS Date32) = data.date -03)----TableScan: data projection=[date, ticker, time] +02)--TableScan: data projection=[date, ticker, time], unsupported_filters=[data.ticker = Utf8View("A"), CAST(data.time AS Date32) = data.date] physical_plan 01)SortPreservingMergeExec: [date@0 ASC NULLS LAST] 02)--FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 @@ -82,8 +79,7 @@ ORDER BY "ticker" ---- logical_plan 01)Sort: data.ticker ASC NULLS LAST -02)--Filter: data.ticker = Utf8View("A") AND CAST(data.time AS Date32) = data.date -03)----TableScan: data projection=[date, ticker, time] +02)--TableScan: data projection=[date, ticker, time], unsupported_filters=[data.ticker = Utf8View("A"), CAST(data.time AS Date32) = data.date] physical_plan 01)CoalescePartitionsExec 02)--FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 @@ -98,8 +94,7 @@ ORDER BY "time", "date"; ---- logical_plan 01)Sort: data.time ASC NULLS LAST, data.date ASC NULLS LAST -02)--Filter: data.ticker = Utf8View("A") AND CAST(data.time AS Date32) = data.date -03)----TableScan: data projection=[date, ticker, time] +02)--TableScan: data projection=[date, ticker, time], unsupported_filters=[data.ticker = Utf8View("A"), CAST(data.time AS Date32) = data.date] physical_plan 01)SortPreservingMergeExec: [time@2 ASC NULLS LAST, date@0 ASC NULLS LAST] 02)--FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 @@ -115,8 +110,7 @@ ORDER BY "time" ---- logical_plan 01)Sort: data.time ASC NULLS LAST -02)--Filter: data.ticker = Utf8View("A") AND CAST(data.time AS Date32) != data.date -03)----TableScan: data projection=[date, ticker, time] +02)--TableScan: data projection=[date, ticker, time], unsupported_filters=[data.ticker = Utf8View("A"), CAST(data.time AS Date32) != data.date] # no relation between time & date # should also be pipeline breaking @@ -127,8 +121,7 @@ ORDER BY "time" ---- logical_plan 01)Sort: data.time ASC NULLS LAST -02)--Filter: data.ticker = Utf8View("A") -03)----TableScan: data projection=[date, ticker, time] +02)--TableScan: data projection=[date, ticker, time], unsupported_filters=[data.ticker = Utf8View("A")] # query query TT @@ -138,8 +131,7 @@ ORDER BY "ticker", "time"; ---- logical_plan 01)Sort: data.ticker ASC NULLS LAST, data.time ASC NULLS LAST -02)--Filter: data.date = Date32("2006-01-02") -03)----TableScan: data projection=[date, ticker, time] +02)--TableScan: data projection=[date, ticker, time], unsupported_filters=[data.date = Date32("2006-01-02")] physical_plan 01)SortPreservingMergeExec: [ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] 02)--FilterExec: date@0 = 2006-01-02 diff --git a/datafusion/sqllogictest/test_files/grouping_set_repartition.slt b/datafusion/sqllogictest/test_files/grouping_set_repartition.slt index 16ab90651c8b3..ca46cc70a9fee 100644 --- a/datafusion/sqllogictest/test_files/grouping_set_repartition.slt +++ b/datafusion/sqllogictest/test_files/grouping_set_repartition.slt @@ -126,18 +126,15 @@ logical_plan 06)----------Projection: Utf8("store") AS channel, sales.brand, sum(sales.amount) AS total 07)------------Aggregate: groupBy=[[sales.brand]], aggr=[[sum(CAST(sales.amount AS Int64))]] 08)--------------Projection: sales.brand, sales.amount -09)----------------Filter: sales.channel = Utf8View("store") -10)------------------TableScan: sales projection=[channel, brand, amount], partial_filters=[sales.channel = Utf8View("store")] -11)----------Projection: Utf8("web") AS channel, sales.brand, sum(sales.amount) AS total -12)------------Aggregate: groupBy=[[sales.brand]], aggr=[[sum(CAST(sales.amount AS Int64))]] -13)--------------Projection: sales.brand, sales.amount -14)----------------Filter: sales.channel = Utf8View("web") -15)------------------TableScan: sales projection=[channel, brand, amount], partial_filters=[sales.channel = Utf8View("web")] -16)----------Projection: Utf8("catalog") AS channel, sales.brand, sum(sales.amount) AS total -17)------------Aggregate: groupBy=[[sales.brand]], aggr=[[sum(CAST(sales.amount AS Int64))]] -18)--------------Projection: sales.brand, sales.amount -19)----------------Filter: sales.channel = Utf8View("catalog") -20)------------------TableScan: sales projection=[channel, brand, amount], partial_filters=[sales.channel = Utf8View("catalog")] +09)----------------TableScan: sales projection=[brand, amount, channel], partial_filters=[sales.channel = Utf8View("store")] +10)----------Projection: Utf8("web") AS channel, sales.brand, sum(sales.amount) AS total +11)------------Aggregate: groupBy=[[sales.brand]], aggr=[[sum(CAST(sales.amount AS Int64))]] +12)--------------Projection: sales.brand, sales.amount +13)----------------TableScan: sales projection=[brand, amount, channel], partial_filters=[sales.channel = Utf8View("web")] +14)----------Projection: Utf8("catalog") AS channel, sales.brand, sum(sales.amount) AS total +15)------------Aggregate: groupBy=[[sales.brand]], aggr=[[sum(CAST(sales.amount AS Int64))]] +16)--------------Projection: sales.brand, sales.amount +17)----------------TableScan: sales projection=[brand, amount, channel], partial_filters=[sales.channel = Utf8View("catalog")] physical_plan 01)SortPreservingMergeExec: [channel@0 ASC, brand@1 ASC] 02)--SortExec: expr=[channel@0 ASC, brand@1 ASC], preserve_partitioning=[true] @@ -150,20 +147,20 @@ physical_plan 09)----------------AggregateExec: mode=FinalPartitioned, gby=[brand@0 as brand], aggr=[sum(sales.amount)] 10)------------------RepartitionExec: partitioning=Hash([brand@0], 4), input_partitions=4 11)--------------------AggregateExec: mode=Partial, gby=[brand@0 as brand], aggr=[sum(sales.amount)] -12)----------------------FilterExec: channel@0 = store, projection=[brand@1, amount@2] -13)------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=1/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=2/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=3/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=4/data.parquet]]}, projection=[channel, brand, amount], file_type=parquet, predicate=channel@0 = store, pruning_predicate=channel_null_count@2 != row_count@3 AND channel_min@0 <= store AND store <= channel_max@1, required_guarantees=[channel in (store)] +12)----------------------FilterExec: channel@2 = store, projection=[brand@0, amount@1] +13)------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=1/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=2/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=3/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=4/data.parquet]]}, projection=[brand, amount, channel], file_type=parquet, predicate=channel@0 = store, pruning_predicate=channel_null_count@2 != row_count@3 AND channel_min@0 <= store AND store <= channel_max@1, required_guarantees=[channel in (store)] 14)--------------ProjectionExec: expr=[web as channel, brand@0 as brand, sum(sales.amount)@1 as total] 15)----------------AggregateExec: mode=FinalPartitioned, gby=[brand@0 as brand], aggr=[sum(sales.amount)] 16)------------------RepartitionExec: partitioning=Hash([brand@0], 4), input_partitions=4 17)--------------------AggregateExec: mode=Partial, gby=[brand@0 as brand], aggr=[sum(sales.amount)] -18)----------------------FilterExec: channel@0 = web, projection=[brand@1, amount@2] -19)------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=1/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=2/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=3/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=4/data.parquet]]}, projection=[channel, brand, amount], file_type=parquet, predicate=channel@0 = web, pruning_predicate=channel_null_count@2 != row_count@3 AND channel_min@0 <= web AND web <= channel_max@1, required_guarantees=[channel in (web)] +18)----------------------FilterExec: channel@2 = web, projection=[brand@0, amount@1] +19)------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=1/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=2/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=3/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=4/data.parquet]]}, projection=[brand, amount, channel], file_type=parquet, predicate=channel@0 = web, pruning_predicate=channel_null_count@2 != row_count@3 AND channel_min@0 <= web AND web <= channel_max@1, required_guarantees=[channel in (web)] 20)--------------ProjectionExec: expr=[catalog as channel, brand@0 as brand, sum(sales.amount)@1 as total] 21)----------------AggregateExec: mode=FinalPartitioned, gby=[brand@0 as brand], aggr=[sum(sales.amount)] 22)------------------RepartitionExec: partitioning=Hash([brand@0], 4), input_partitions=4 23)--------------------AggregateExec: mode=Partial, gby=[brand@0 as brand], aggr=[sum(sales.amount)] -24)----------------------FilterExec: channel@0 = catalog, projection=[brand@1, amount@2] -25)------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=1/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=2/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=3/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=4/data.parquet]]}, projection=[channel, brand, amount], file_type=parquet, predicate=channel@0 = catalog, pruning_predicate=channel_null_count@2 != row_count@3 AND channel_min@0 <= catalog AND catalog <= channel_max@1, required_guarantees=[channel in (catalog)] +24)----------------------FilterExec: channel@2 = catalog, projection=[brand@0, amount@1] +25)------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=1/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=2/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=3/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=4/data.parquet]]}, projection=[brand, amount, channel], file_type=parquet, predicate=channel@0 = catalog, pruning_predicate=channel_null_count@2 != row_count@3 AND channel_min@0 <= catalog AND catalog <= channel_max@1, required_guarantees=[channel in (catalog)] query TTI rowsort SELECT channel, brand, SUM(total) as grand_total diff --git a/datafusion/sqllogictest/test_files/join.slt.part b/datafusion/sqllogictest/test_files/join.slt.part index c0a838c97d552..a280408bef753 100644 --- a/datafusion/sqllogictest/test_files/join.slt.part +++ b/datafusion/sqllogictest/test_files/join.slt.part @@ -693,8 +693,8 @@ query TT explain select * from t1 inner join t2 on true; ---- logical_plan -01)Cross Join: -02)--TableScan: t1 projection=[t1_id, t1_name, t1_int] +01)Cross Join: +02)--TableScan: t1 projection=[t1_id, t1_name, t1_int], unsupported_filters=[Boolean(true)] 03)--TableScan: t2 projection=[t2_id, t2_name, t2_int] physical_plan 01)CrossJoinExec @@ -927,12 +927,11 @@ JOIN department AS d ON (e.name = 'Alice' OR e.name = 'Bob'); ---- logical_plan -01)Cross Join: +01)Cross Join: 02)--SubqueryAlias: e -03)----Filter: employees.name = Utf8View("Alice") OR employees.name = Utf8View("Bob") -04)------TableScan: employees projection=[emp_id, name] -05)--SubqueryAlias: d -06)----TableScan: department projection=[dept_name] +03)----TableScan: employees projection=[emp_id, name], unsupported_filters=[employees.name = Utf8View("Alice") OR employees.name = Utf8View("Bob")] +04)--SubqueryAlias: d +05)----TableScan: department projection=[dept_name] physical_plan 01)CrossJoinExec 02)--FilterExec: name@1 = Alice OR name@1 = Bob @@ -977,10 +976,9 @@ logical_plan 02)--Projection: e.emp_id, e.name, d.dept_name 03)----Left Join: e.emp_id = d.emp_id 04)------SubqueryAlias: e -05)--------Filter: employees.name = Utf8View("Alice") OR employees.name = Utf8View("Carol") -06)----------TableScan: employees projection=[emp_id, name] -07)------SubqueryAlias: d -08)--------TableScan: department projection=[emp_id, dept_name] +05)--------TableScan: employees projection=[emp_id, name], unsupported_filters=[employees.name = Utf8View("Alice") OR employees.name = Utf8View("Carol")] +06)------SubqueryAlias: d +07)--------TableScan: department projection=[emp_id, dept_name] physical_plan 01)FilterExec: dept_name@2 != Engineering AND name@1 = Alice OR name@1 = Carol 02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 @@ -1362,10 +1360,9 @@ logical_plan 01)Inner Join: col1 = CAST(s.b AS Int64) 02)--Inner Join: col0 = CAST(f.a AS Int64) 03)----Projection: CAST(pairs.x AS Int64) + Int64(1) AS col0, CAST(pairs.y AS Int64) + Int64(1) AS col1 -04)------Filter: pairs.y = pairs.x -05)--------TableScan: pairs projection=[x, y] -06)----TableScan: f projection=[a] -07)--TableScan: s projection=[b] +04)------TableScan: pairs projection=[x, y], unsupported_filters=[pairs.y = pairs.x] +05)----TableScan: f projection=[a] +06)--TableScan: s projection=[b] physical_plan 01)ProjectionExec: expr=[col0@1 as col0, col1@2 as col1, a@3 as a, b@0 as b] 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(s.b AS Int64)@1, col1@1)], projection=[b@0, col0@2, col1@3, a@4] diff --git a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt index 1b25a01d62104..99243dd0628f7 100644 --- a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt +++ b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt @@ -88,11 +88,9 @@ logical_plan 02)--Projection: t2.a AS a2, t2.b 03)----RightSemi Join: t1.d = t2.d, t1.c = t2.c 04)------SubqueryAlias: t1 -05)--------Filter: annotated_data.d = Int32(3) -06)----------TableScan: annotated_data projection=[c, d], partial_filters=[annotated_data.d = Int32(3)] -07)------SubqueryAlias: t2 -08)--------Filter: annotated_data.d = Int32(3) -09)----------TableScan: annotated_data projection=[a, b, c, d], partial_filters=[annotated_data.d = Int32(3)] +05)--------TableScan: annotated_data projection=[c, d], partial_filters=[annotated_data.d = Int32(3)] +06)------SubqueryAlias: t2 +07)--------TableScan: annotated_data projection=[a, b, c, d], partial_filters=[annotated_data.d = Int32(3)] physical_plan 01)SortPreservingMergeExec: [a2@0 ASC NULLS LAST, b@1 ASC NULLS LAST], fetch=10 02)--ProjectionExec: expr=[a@0 as a2, b@1 as b] diff --git a/datafusion/sqllogictest/test_files/join_is_not_distinct_from.slt b/datafusion/sqllogictest/test_files/join_is_not_distinct_from.slt index 8246f489c446d..fd94c0cbbf127 100644 --- a/datafusion/sqllogictest/test_files/join_is_not_distinct_from.slt +++ b/datafusion/sqllogictest/test_files/join_is_not_distinct_from.slt @@ -136,9 +136,8 @@ JOIN t2 ON ((t1.val+1) IS NOT DISTINCT FROM (t2.val+1)) AND ((t1.val + 1) IS NOT logical_plan 01)Projection: t1.id AS t1_id, t2.id AS t2_id, t1.val, t2.val 02)--Inner Join: CAST(t1.val AS Int64) + Int64(1) = CAST(t2.val AS Int64) + Int64(1) -03)----Filter: CAST(t1.val AS Int64) + Int64(1) IS NOT DISTINCT FROM Int64(11) -04)------TableScan: t1 projection=[id, val] -05)----TableScan: t2 projection=[id, val] +03)----TableScan: t1 projection=[id, val], unsupported_filters=[CAST(t1.val AS Int64) + Int64(1) IS NOT DISTINCT FROM Int64(11)] +04)----TableScan: t2 projection=[id, val] physical_plan 01)ProjectionExec: expr=[id@0 as t1_id, id@2 as t2_id, val@1 as val, val@3 as val] 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1.val + Int64(1)@2, t2.val + Int64(1)@2)], projection=[id@0, val@1, id@3, val@4], NullsEqual: true diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index df3cad1a141c8..1057deb1aab78 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -1052,10 +1052,8 @@ WHERE join_t2.t2_id < 100 ---- logical_plan 01)Inner Join: join_t1.t1_id = join_t2.t2_id -02)--Filter: join_t1.t1_id < UInt32(100) -03)----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] -04)--Filter: join_t2.t2_id < UInt32(100) -05)----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int], unsupported_filters=[join_t1.t1_id < UInt32(100)] +03)--TableScan: join_t2 projection=[t2_id, t2_name, t2_int], unsupported_filters=[join_t2.t2_id < UInt32(100)] # Reduce left join 2 (to inner join) @@ -1069,8 +1067,7 @@ WHERE join_t2.t2_int < 10 or (join_t1.t1_int > 2 and join_t2.t2_name != 'w') logical_plan 01)Inner Join: join_t1.t1_id = join_t2.t2_id Filter: join_t2.t2_int < UInt32(10) OR join_t1.t1_int > UInt32(2) AND join_t2.t2_name != Utf8View("w") 02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] -03)--Filter: join_t2.t2_int < UInt32(10) OR join_t2.t2_name != Utf8View("w") -04)----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] +03)--TableScan: join_t2 projection=[t2_id, t2_name, t2_int], unsupported_filters=[join_t2.t2_int < UInt32(10) OR join_t2.t2_name != Utf8View("w")] # Reduce left join 3 (to inner join) @@ -1091,12 +1088,10 @@ logical_plan 02)--SubqueryAlias: t3 03)----Projection: join_t1.t1_id, join_t1.t1_name, join_t1.t1_int 04)------Inner Join: join_t1.t1_id = join_t2.t2_id -05)--------Filter: join_t1.t1_id < UInt32(100) -06)----------TableScan: join_t1 projection=[t1_id, t1_name, t1_int] -07)--------Projection: join_t2.t2_id -08)----------Filter: join_t2.t2_int < UInt32(3) AND join_t2.t2_id < UInt32(100) -09)------------TableScan: join_t2 projection=[t2_id, t2_int] -10)--TableScan: join_t2 projection=[t2_id, t2_name, t2_int] +05)--------TableScan: join_t1 projection=[t1_id, t1_name, t1_int], unsupported_filters=[join_t1.t1_id < UInt32(100)] +06)--------Projection: join_t2.t2_id +07)----------TableScan: join_t2 projection=[t2_id, t2_int], unsupported_filters=[join_t2.t2_int < UInt32(3), join_t2.t2_id < UInt32(100)] +08)--TableScan: join_t2 projection=[t2_id, t2_name, t2_int] # Reduce right join 1 (to inner join) @@ -1109,9 +1104,8 @@ WHERE join_t1.t1_int IS NOT NULL ---- logical_plan 01)Inner Join: join_t1.t1_id = join_t2.t2_id -02)--Filter: join_t1.t1_int IS NOT NULL -03)----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] -04)--TableScan: join_t2 projection=[t2_id, t2_name, t2_int] +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int], unsupported_filters=[join_t1.t1_int IS NOT NULL] +03)--TableScan: join_t2 projection=[t2_id, t2_name, t2_int] # Reduce right join 2 (to inner join) @@ -1139,8 +1133,7 @@ WHERE join_t2.t2_name IS NOT NULL logical_plan 01)Right Join: join_t1.t1_id = join_t2.t2_id 02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] -03)--Filter: join_t2.t2_name IS NOT NULL -04)----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] +03)--TableScan: join_t2 projection=[t2_id, t2_name, t2_int], unsupported_filters=[join_t2.t2_name IS NOT NULL] # Reduce full join to left join @@ -1153,9 +1146,8 @@ WHERE join_t1.t1_name != 'b' ---- logical_plan 01)Left Join: join_t1.t1_id = join_t2.t2_id -02)--Filter: join_t1.t1_name != Utf8View("b") -03)----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] -04)--TableScan: join_t2 projection=[t2_id, t2_name, t2_int] +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int], unsupported_filters=[join_t1.t1_name != Utf8View("b")] +03)--TableScan: join_t2 projection=[t2_id, t2_name, t2_int] # Reduce full join to inner join @@ -1168,10 +1160,8 @@ WHERE join_t1.t1_name != 'b' and join_t2.t2_name = 'x' ---- logical_plan 01)Inner Join: join_t1.t1_id = join_t2.t2_id -02)--Filter: join_t1.t1_name != Utf8View("b") -03)----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] -04)--Filter: join_t2.t2_name = Utf8View("x") -05)----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int], unsupported_filters=[join_t1.t1_name != Utf8View("b")] +03)--TableScan: join_t2 projection=[t2_id, t2_name, t2_int], unsupported_filters=[join_t2.t2_name = Utf8View("x")] ### # Configuration teardown @@ -1222,8 +1212,7 @@ logical_plan 01)LeftSemi Join: lsaj_t1.t1_id = lsaj_t2.t2_id 02)--TableScan: lsaj_t1 projection=[t1_id, t1_name] 03)--Projection: lsaj_t2.t2_id -04)----Filter: lsaj_t2.t2_int > UInt32(1) -05)------TableScan: lsaj_t2 projection=[t2_id, t2_int] +04)----TableScan: lsaj_t2 projection=[t2_id, t2_int], unsupported_filters=[lsaj_t2.t2_int > UInt32(1)] # Left anti join @@ -1847,8 +1836,7 @@ logical_plan 02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] 03)--SubqueryAlias: __correlated_sq_1 04)----Projection: CAST(join_t2.t2_id AS Int64) + Int64(1), join_t2.t2_int -05)------Filter: join_t2.t2_int > UInt32(0) -06)--------TableScan: join_t2 projection=[t2_id, t2_int] +05)------TableScan: join_t2 projection=[t2_id, t2_int], unsupported_filters=[join_t2.t2_int > UInt32(0)] query ITI rowsort select join_t1.t1_id, join_t1.t1_name, join_t1.t1_int @@ -1883,8 +1871,7 @@ logical_plan 02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] 03)--SubqueryAlias: __correlated_sq_1 04)----Projection: CAST(join_t2.t2_id AS Int64) + Int64(1), join_t2.t2_int, join_t2.t2_name -05)------Filter: join_t2.t2_int > UInt32(0) -06)--------TableScan: join_t2 projection=[t2_id, t2_name, t2_int] +05)------TableScan: join_t2 projection=[t2_id, t2_name, t2_int], unsupported_filters=[join_t2.t2_int > UInt32(0)] query ITI rowsort select join_t1.t1_id, join_t1.t1_name, join_t1.t1_int @@ -1912,11 +1899,10 @@ where join_t1.t1_id + 12 in ---- logical_plan 01)LeftSemi Join: CAST(join_t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.join_t2.t2_id + Int64(1) -02)--Filter: join_t1.t1_int > UInt32(0) -03)----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] -04)--SubqueryAlias: __correlated_sq_1 -05)----Projection: CAST(join_t2.t2_id AS Int64) + Int64(1) -06)------TableScan: join_t2 projection=[t2_id] +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int], unsupported_filters=[join_t1.t1_int > UInt32(0)] +03)--SubqueryAlias: __correlated_sq_1 +04)----Projection: CAST(join_t2.t2_id AS Int64) + Int64(1) +05)------TableScan: join_t2 projection=[t2_id] # Not in subquery to join with correlated outer filter @@ -1951,11 +1937,10 @@ where join_t1.t1_id + 12 in ---- logical_plan 01)LeftSemi Join: CAST(join_t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.join_t2.t2_id + Int64(1) Filter: join_t1.t1_int <= __correlated_sq_1.t2_int AND join_t1.t1_name != __correlated_sq_1.t2_name -02)--Filter: join_t1.t1_id > UInt32(0) -03)----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] -04)--SubqueryAlias: __correlated_sq_1 -05)----Projection: CAST(join_t2.t2_id AS Int64) + Int64(1), join_t2.t2_int, join_t2.t2_name -06)------TableScan: join_t2 projection=[t2_id, t2_name, t2_int] +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int], unsupported_filters=[join_t1.t1_id > UInt32(0)] +03)--SubqueryAlias: __correlated_sq_1 +04)----Projection: CAST(join_t2.t2_id AS Int64) + Int64(1), join_t2.t2_int, join_t2.t2_name +05)------TableScan: join_t2 projection=[t2_id, t2_name, t2_int] query ITI rowsort select join_t1.t1_id, join_t1.t1_name, join_t1.t1_int @@ -1985,14 +1970,13 @@ where join_t1.t1_id + 12 in (select join_t2.t2_id + 1 from join_t2) logical_plan 01)LeftSemi Join: CAST(join_t1.t1_int AS Int64) = __correlated_sq_2.join_t2.t2_int + Int64(1) 02)--LeftSemi Join: CAST(join_t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.join_t2.t2_id + Int64(1) -03)----Filter: join_t1.t1_id > UInt32(0) -04)------TableScan: join_t1 projection=[t1_id, t1_name, t1_int] -05)----SubqueryAlias: __correlated_sq_1 -06)------Projection: CAST(join_t2.t2_id AS Int64) + Int64(1) -07)--------TableScan: join_t2 projection=[t2_id] -08)--SubqueryAlias: __correlated_sq_2 -09)----Projection: CAST(join_t2.t2_int AS Int64) + Int64(1) -10)------TableScan: join_t2 projection=[t2_int] +03)----TableScan: join_t1 projection=[t1_id, t1_name, t1_int], unsupported_filters=[join_t1.t1_id > UInt32(0)] +04)----SubqueryAlias: __correlated_sq_1 +05)------Projection: CAST(join_t2.t2_id AS Int64) + Int64(1) +06)--------TableScan: join_t2 projection=[t2_id] +07)--SubqueryAlias: __correlated_sq_2 +08)----Projection: CAST(join_t2.t2_int AS Int64) + Int64(1) +09)------TableScan: join_t2 projection=[t2_int] query ITI select join_t1.t1_id, join_t1.t1_name, join_t1.t1_int @@ -2026,11 +2010,9 @@ WHERE join_t1.t1_id > 10 AND join_t2.t2_int > 1 ---- logical_plan 01)Inner Join: Filter: join_t1.t1_id > join_t2.t2_id -02)--Filter: join_t1.t1_id > UInt32(10) -03)----TableScan: join_t1 projection=[t1_id] -04)--Projection: join_t2.t2_id -05)----Filter: join_t2.t2_int > UInt32(1) -06)------TableScan: join_t2 projection=[t2_id, t2_int] +02)--TableScan: join_t1 projection=[t1_id], unsupported_filters=[join_t1.t1_id > UInt32(10)] +03)--Projection: join_t2.t2_id +04)----TableScan: join_t2 projection=[t2_id, t2_int], unsupported_filters=[join_t2.t2_int > UInt32(1)] physical_plan 01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id] 02)--NestedLoopJoinExec: join_type=Inner, filter=t1_id@0 > t2_id@1 @@ -2065,11 +2047,9 @@ RIGHT JOIN (select t2_id from join_t2 where join_t2.t2_id > 11) as join_t2 logical_plan 01)Right Join: Filter: join_t1.t1_id < join_t2.t2_id 02)--SubqueryAlias: join_t1 -03)----Filter: join_t1.t1_id > UInt32(22) -04)------TableScan: join_t1 projection=[t1_id] -05)--SubqueryAlias: join_t2 -06)----Filter: join_t2.t2_id > UInt32(11) -07)------TableScan: join_t2 projection=[t2_id] +03)----TableScan: join_t1 projection=[t1_id], unsupported_filters=[join_t1.t1_id > UInt32(22)] +04)--SubqueryAlias: join_t2 +05)----TableScan: join_t2 projection=[t2_id], unsupported_filters=[join_t2.t2_id > UInt32(11)] physical_plan 01)NestedLoopJoinExec: join_type=Right, filter=t1_id@0 < t2_id@1 02)--CoalescePartitionsExec @@ -2172,8 +2152,7 @@ logical_plan 02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] 03)--SubqueryAlias: __correlated_sq_1 04)----Projection: join_t2.t2_id -05)------Filter: join_t2.t2_int < UInt32(3) -06)--------TableScan: join_t2 projection=[t2_id, t2_int] +05)------TableScan: join_t2 projection=[t2_id, t2_int], unsupported_filters=[join_t2.t2_int < UInt32(3)] statement ok set datafusion.optimizer.repartition_joins = false; @@ -2217,10 +2196,9 @@ WHERE EXISTS ( ---- logical_plan 01)LeftSemi Join: Filter: CAST(join_t1.t1_id AS Int64) + Int64(1) > CAST(__correlated_sq_1.t2_id AS Int64) * Int64(2) -02)--Filter: join_t1.t1_int < UInt32(3) -03)----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] -04)--SubqueryAlias: __correlated_sq_1 -05)----TableScan: join_t2 projection=[t2_id] +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int], unsupported_filters=[join_t1.t1_int < UInt32(3)] +03)--SubqueryAlias: __correlated_sq_1 +04)----TableScan: join_t2 projection=[t2_id] statement ok set datafusion.optimizer.repartition_joins = false; @@ -4043,15 +4021,14 @@ logical_plan 07)------------TableScan: sales_global projection=[ts, sn, amount, currency] 08)----------SubqueryAlias: e 09)------------Projection: exchange_rates.ts, exchange_rates.currency_from, exchange_rates.rate -10)--------------Filter: exchange_rates.currency_to = Utf8View("USD") -11)----------------TableScan: exchange_rates projection=[ts, currency_from, currency_to, rate] +10)--------------TableScan: exchange_rates projection=[ts, currency_from, rate, currency_to], unsupported_filters=[exchange_rates.currency_to = Utf8View("USD")] physical_plan 01)SortExec: expr=[sn@1 ASC NULLS LAST], preserve_partitioning=[false] 02)--ProjectionExec: expr=[ts@1 as ts, sn@0 as sn, amount@2 as amount, currency@3 as currency, CAST(amount@2 AS Float32) * last_value(e.rate)@4 as amount_usd] 03)----AggregateExec: mode=Single, gby=[sn@1 as sn, ts@0 as ts, amount@2 as amount, currency@3 as currency], aggr=[last_value(e.rate)] 04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(currency@3, currency_from@1)], filter=ts@0 >= ts@1, projection=[ts@0, sn@1, amount@2, currency@3, rate@6] 05)--------DataSourceExec: partitions=1, partition_sizes=[0] -06)--------FilterExec: currency_to@2 = USD, projection=[ts@0, currency_from@1, rate@3] +06)--------FilterExec: currency_to@3 = USD, projection=[ts@0, currency_from@1, rate@2] 07)----------DataSourceExec: partitions=1, partition_sizes=[0] statement ok @@ -4392,8 +4369,7 @@ logical_plan 03)----TableScan: test projection=[a, b, c] 04)----SubqueryAlias: __correlated_sq_1 05)------Projection: test.a -06)--------Filter: test.b > Int32(3) -07)----------TableScan: test projection=[a, b] +06)--------TableScan: test projection=[a, b], unsupported_filters=[test.b > Int32(3)] physical_plan 01)SortPreservingMergeExec: [c@2 DESC] 02)--HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(a@0, a@0)] @@ -4412,8 +4388,7 @@ logical_plan 03)----TableScan: test projection=[a, b, c] 04)----SubqueryAlias: __correlated_sq_1 05)------Projection: test.a -06)--------Filter: test.b > Int32(3) -07)----------TableScan: test projection=[a, b] +06)--------TableScan: test projection=[a, b], unsupported_filters=[test.b > Int32(3)] physical_plan 01)SortPreservingMergeExec: [c@2 DESC NULLS LAST] 02)--HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(a@0, a@0)] @@ -4578,20 +4553,19 @@ query TT explain SELECT j1_string, j2_string FROM j1, LATERAL (SELECT * FROM j2 WHERE j1_id < j2_id) AS j2; ---- logical_plan -01)Cross Join: +01)Cross Join: 02)--TableScan: j1 projection=[j1_string] 03)--SubqueryAlias: j2 04)----Projection: j2.j2_string 05)------Subquery: -06)--------Filter: outer_ref(j1.j1_id) < j2.j2_id -07)----------TableScan: j2 projection=[j2_string, j2_id] +06)--------TableScan: j2 projection=[j2_string, j2_id], unsupported_filters=[outer_ref(j1.j1_id) < j2.j2_id] physical_plan_error This feature is not implemented: Physical plan does not support logical expression OuterReferenceColumn(Field { name: "j1_id", data_type: Int32, nullable: true }, Column { relation: Some(Bare { table: "j1" }), name: "j1_id" }) query TT explain SELECT * FROM j1 JOIN (j2 JOIN j3 ON(j2_id = j3_id - 2)) ON(j1_id = j2_id), LATERAL (SELECT * FROM j3 WHERE j3_string = j2_string) as j4 ---- logical_plan -01)Cross Join: +01)Cross Join: 02)--Inner Join: CAST(j2.j2_id AS Int64) = CAST(j3.j3_id AS Int64) - Int64(2) 03)----Inner Join: j1.j1_id = j2.j2_id 04)------TableScan: j1 projection=[j1_string, j1_id] @@ -4599,51 +4573,47 @@ logical_plan 06)----TableScan: j3 projection=[j3_string, j3_id] 07)--SubqueryAlias: j4 08)----Subquery: -09)------Filter: j3.j3_string = outer_ref(j2.j2_string) -10)--------TableScan: j3 projection=[j3_string, j3_id] +09)------TableScan: j3 projection=[j3_string, j3_id], unsupported_filters=[j3.j3_string = outer_ref(j2.j2_string)] physical_plan_error This feature is not implemented: Physical plan does not support logical expression OuterReferenceColumn(Field { name: "j2_string", data_type: Utf8View, nullable: true }, Column { relation: Some(Bare { table: "j2" }), name: "j2_string" }) query TT explain SELECT * FROM j1, LATERAL (SELECT * FROM j1, LATERAL (SELECT * FROM j2 WHERE j1_id = j2_id) as j2) as j2; ---- logical_plan -01)Cross Join: +01)Cross Join: 02)--TableScan: j1 projection=[j1_string, j1_id] 03)--SubqueryAlias: j2 04)----Subquery: -05)------Cross Join: +05)------Cross Join: 06)--------TableScan: j1 projection=[j1_string, j1_id] 07)--------SubqueryAlias: j2 08)----------Subquery: -09)------------Filter: outer_ref(j1.j1_id) = j2.j2_id -10)--------------TableScan: j2 projection=[j2_string, j2_id] +09)------------TableScan: j2 projection=[j2_string, j2_id], unsupported_filters=[outer_ref(j1.j1_id) = j2.j2_id] physical_plan_error This feature is not implemented: Physical plan does not support logical expression OuterReferenceColumn(Field { name: "j1_id", data_type: Int32, nullable: true }, Column { relation: Some(Bare { table: "j1" }), name: "j1_id" }) query TT explain SELECT j1_string, j2_string FROM j1 LEFT JOIN LATERAL (SELECT * FROM j2 WHERE j1_id < j2_id) AS j2 ON(true); ---- logical_plan -01)Left Join: +01)Left Join: 02)--TableScan: j1 projection=[j1_string] 03)--SubqueryAlias: j2 04)----Projection: j2.j2_string 05)------Subquery: -06)--------Filter: outer_ref(j1.j1_id) < j2.j2_id -07)----------TableScan: j2 projection=[j2_string, j2_id] +06)--------TableScan: j2 projection=[j2_string, j2_id], unsupported_filters=[outer_ref(j1.j1_id) < j2.j2_id] physical_plan_error This feature is not implemented: Physical plan does not support logical expression OuterReferenceColumn(Field { name: "j1_id", data_type: Int32, nullable: true }, Column { relation: Some(Bare { table: "j1" }), name: "j1_id" }) query TT explain SELECT * FROM j1, (j2 LEFT JOIN LATERAL (SELECT * FROM j3 WHERE j1_id + j2_id = j3_id) AS j3 ON(true)); ---- logical_plan -01)Cross Join: +01)Cross Join: 02)--TableScan: j1 projection=[j1_string, j1_id] -03)--Left Join: +03)--Left Join: 04)----TableScan: j2 projection=[j2_string, j2_id] 05)----SubqueryAlias: j3 06)------Subquery: -07)--------Filter: outer_ref(j1.j1_id) + outer_ref(j2.j2_id) = j3.j3_id -08)----------TableScan: j3 projection=[j3_string, j3_id] +07)--------TableScan: j3 projection=[j3_string, j3_id], unsupported_filters=[outer_ref(j1.j1_id) + outer_ref(j2.j2_id) = j3.j3_id] physical_plan_error This feature is not implemented: Physical plan does not support logical expression OuterReferenceColumn(Field { name: "j1_id", data_type: Int32, nullable: true }, Column { relation: Some(Bare { table: "j1" }), name: "j1_id" }) query TT diff --git a/datafusion/sqllogictest/test_files/json.slt b/datafusion/sqllogictest/test_files/json.slt index b46b8c49d6623..e35e9362e8c0b 100644 --- a/datafusion/sqllogictest/test_files/json.slt +++ b/datafusion/sqllogictest/test_files/json.slt @@ -144,5 +144,7 @@ SELECT id FROM json_partitioned_test WHERE part = 1 ORDER BY id query TT EXPLAIN SELECT id FROM json_partitioned_test WHERE part = 2 ---- -logical_plan TableScan: json_partitioned_test projection=[id], full_filters=[json_partitioned_test.part = Int32(2)] +logical_plan +01)Projection: json_partitioned_test.id +02)--TableScan: json_partitioned_test projection=[id, part], full_filters=[json_partitioned_test.part = Int32(2)] physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/partitioned_table_json/part=2/data.json]]}, projection=[id], file_type=json diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index 429181a2d385b..09478cc0c10ae 100644 --- a/datafusion/sqllogictest/test_files/limit.slt +++ b/datafusion/sqllogictest/test_files/limit.slt @@ -367,8 +367,7 @@ logical_plan 02)--Aggregate: groupBy=[[]], aggr=[[count(Int64(1))]] 03)----Projection: 04)------Limit: skip=6, fetch=3 -05)--------Filter: t1.a > Int32(3) -06)----------TableScan: t1 projection=[a] +05)--------TableScan: t1 projection=[a], unsupported_filters=[t1.a > Int32(3)], fetch=9 physical_plan 01)ProjectionExec: expr=[count(Int64(1))@0 as count(*)] 02)--AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] @@ -871,4 +870,4 @@ DROP TABLE test_limit_with_partitions; # Tear down src_table table: statement ok -DROP TABLE src_table; \ No newline at end of file +DROP TABLE src_table; diff --git a/datafusion/sqllogictest/test_files/map.slt b/datafusion/sqllogictest/test_files/map.slt index 7ea54464d3e99..33446c655a719 100644 --- a/datafusion/sqllogictest/test_files/map.slt +++ b/datafusion/sqllogictest/test_files/map.slt @@ -109,9 +109,7 @@ select * from table_with_map where int_field > 0; query TT explain select * from table_with_map where int_field > 0; ---- -logical_plan -01)Filter: table_with_map.int_field > Int64(0) -02)--TableScan: table_with_map projection=[int_field, map_field] +logical_plan TableScan: table_with_map projection=[int_field, map_field], unsupported_filters=[table_with_map.int_field > Int64(0)] physical_plan 01)FilterExec: int_field@0 > 0 02)--DataSourceExec: partitions=1, partition_sizes=[0] diff --git a/datafusion/sqllogictest/test_files/options.slt b/datafusion/sqllogictest/test_files/options.slt index 0d1583dbc0086..2568052b8ad94 100644 --- a/datafusion/sqllogictest/test_files/options.slt +++ b/datafusion/sqllogictest/test_files/options.slt @@ -26,9 +26,7 @@ create table a(c0 int) as values (1), (2); query TT explain SELECT * FROM a WHERE c0 < 1; ---- -logical_plan -01)Filter: a.c0 < Int32(1) -02)--TableScan: a projection=[c0] +logical_plan TableScan: a projection=[c0], unsupported_filters=[a.c0 < Int32(1)] physical_plan 01)FilterExec: c0@0 < 1 02)--DataSourceExec: partitions=1, partition_sizes=[1] @@ -44,9 +42,7 @@ set datafusion.execution.coalesce_batches = false query TT explain SELECT * FROM a WHERE c0 < 1; ---- -logical_plan -01)Filter: a.c0 < Int32(1) -02)--TableScan: a projection=[c0] +logical_plan TableScan: a projection=[c0], unsupported_filters=[a.c0 < Int32(1)] physical_plan 01)FilterExec: c0@0 < 1 02)--DataSourceExec: partitions=1, partition_sizes=[1] @@ -66,9 +62,7 @@ set datafusion.execution.batch_size = 1234; query TT explain SELECT * FROM a WHERE c0 < 1; ---- -logical_plan -01)Filter: a.c0 < Int32(1) -02)--TableScan: a projection=[c0] +logical_plan TableScan: a projection=[c0], unsupported_filters=[a.c0 < Int32(1)] physical_plan 01)FilterExec: c0@0 < 1 02)--DataSourceExec: partitions=1, partition_sizes=[1] diff --git a/datafusion/sqllogictest/test_files/parquet.slt b/datafusion/sqllogictest/test_files/parquet.slt index be713b963b451..0fe9365320557 100644 --- a/datafusion/sqllogictest/test_files/parquet.slt +++ b/datafusion/sqllogictest/test_files/parquet.slt @@ -454,9 +454,7 @@ EXPLAIN largebinary_col LIKE '%a%' AND binaryview_col LIKE '%a%'; ---- -logical_plan -01)Filter: CAST(binary_as_string_default.binary_col AS Utf8View) LIKE Utf8View("%a%") AND CAST(binary_as_string_default.largebinary_col AS Utf8View) LIKE Utf8View("%a%") AND CAST(binary_as_string_default.binaryview_col AS Utf8View) LIKE Utf8View("%a%") -02)--TableScan: binary_as_string_default projection=[binary_col, largebinary_col, binaryview_col], partial_filters=[CAST(binary_as_string_default.binary_col AS Utf8View) LIKE Utf8View("%a%"), CAST(binary_as_string_default.largebinary_col AS Utf8View) LIKE Utf8View("%a%"), CAST(binary_as_string_default.binaryview_col AS Utf8View) LIKE Utf8View("%a%")] +logical_plan TableScan: binary_as_string_default projection=[binary_col, largebinary_col, binaryview_col], partial_filters=[CAST(binary_as_string_default.binary_col AS Utf8View) LIKE Utf8View("%a%"), CAST(binary_as_string_default.largebinary_col AS Utf8View) LIKE Utf8View("%a%"), CAST(binary_as_string_default.binaryview_col AS Utf8View) LIKE Utf8View("%a%")] physical_plan 01)FilterExec: CAST(binary_col@0 AS Utf8View) LIKE %a% AND CAST(largebinary_col@1 AS Utf8View) LIKE %a% AND CAST(binaryview_col@2 AS Utf8View) LIKE %a% 02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -501,9 +499,7 @@ EXPLAIN largebinary_col LIKE '%a%' AND binaryview_col LIKE '%a%'; ---- -logical_plan -01)Filter: binary_as_string_option.binary_col LIKE Utf8View("%a%") AND binary_as_string_option.largebinary_col LIKE Utf8View("%a%") AND binary_as_string_option.binaryview_col LIKE Utf8View("%a%") -02)--TableScan: binary_as_string_option projection=[binary_col, largebinary_col, binaryview_col], partial_filters=[binary_as_string_option.binary_col LIKE Utf8View("%a%"), binary_as_string_option.largebinary_col LIKE Utf8View("%a%"), binary_as_string_option.binaryview_col LIKE Utf8View("%a%")] +logical_plan TableScan: binary_as_string_option projection=[binary_col, largebinary_col, binaryview_col], partial_filters=[binary_as_string_option.binary_col LIKE Utf8View("%a%"), binary_as_string_option.largebinary_col LIKE Utf8View("%a%"), binary_as_string_option.binaryview_col LIKE Utf8View("%a%")] physical_plan 01)FilterExec: binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% 02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -551,9 +547,7 @@ EXPLAIN largebinary_col LIKE '%a%' AND binaryview_col LIKE '%a%'; ---- -logical_plan -01)Filter: binary_as_string_both.binary_col LIKE Utf8View("%a%") AND binary_as_string_both.largebinary_col LIKE Utf8View("%a%") AND binary_as_string_both.binaryview_col LIKE Utf8View("%a%") -02)--TableScan: binary_as_string_both projection=[binary_col, largebinary_col, binaryview_col], partial_filters=[binary_as_string_both.binary_col LIKE Utf8View("%a%"), binary_as_string_both.largebinary_col LIKE Utf8View("%a%"), binary_as_string_both.binaryview_col LIKE Utf8View("%a%")] +logical_plan TableScan: binary_as_string_both projection=[binary_col, largebinary_col, binaryview_col], partial_filters=[binary_as_string_both.binary_col LIKE Utf8View("%a%"), binary_as_string_both.largebinary_col LIKE Utf8View("%a%"), binary_as_string_both.binaryview_col LIKE Utf8View("%a%")] physical_plan 01)FilterExec: binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% 02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -665,9 +659,7 @@ location 'test_files/scratch/parquet/foo.parquet'; query TT explain select * from foo where starts_with(column1, 'f'); ---- -logical_plan -01)Filter: foo.column1 LIKE Utf8View("f%") -02)--TableScan: foo projection=[column1], partial_filters=[foo.column1 LIKE Utf8View("f%")] +logical_plan TableScan: foo projection=[column1], partial_filters=[foo.column1 LIKE Utf8View("f%")] physical_plan 01)FilterExec: column1@0 LIKE f% 02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 diff --git a/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt b/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt index aa94e2e2f2c04..354a4052acffb 100644 --- a/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt +++ b/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt @@ -90,8 +90,7 @@ EXPLAIN select a from t where b > 2 ORDER BY a; logical_plan 01)Sort: t.a ASC NULLS LAST 02)--Projection: t.a -03)----Filter: t.b > Int32(2) -04)------TableScan: t projection=[a, b], partial_filters=[t.b > Int32(2)] +03)----TableScan: t projection=[a, b], partial_filters=[t.b > Int32(2)] physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] @@ -105,8 +104,7 @@ EXPLAIN select a from t_pushdown where b > 2 ORDER BY a; logical_plan 01)Sort: t_pushdown.a ASC NULLS LAST 02)--Projection: t_pushdown.a -03)----Filter: t_pushdown.b > Int32(2) -04)------TableScan: t_pushdown projection=[a, b], partial_filters=[t_pushdown.b > Int32(2)] +03)----TableScan: t_pushdown projection=[a, b], partial_filters=[t_pushdown.b > Int32(2)] physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] @@ -128,8 +126,7 @@ EXPLAIN select a from t where b = 2 ORDER BY b; logical_plan 01)Projection: t.a 02)--Sort: t.b ASC NULLS LAST -03)----Filter: t.b = Int32(2) -04)------TableScan: t projection=[a, b], partial_filters=[t.b = Int32(2)] +03)----TableScan: t projection=[a, b], partial_filters=[t.b = Int32(2)] physical_plan 01)CoalescePartitionsExec 02)--FilterExec: b@1 = 2, projection=[a@0] @@ -142,8 +139,7 @@ EXPLAIN select a from t_pushdown where b = 2 ORDER BY b; logical_plan 01)Projection: t_pushdown.a 02)--Sort: t_pushdown.b ASC NULLS LAST -03)----Filter: t_pushdown.b = Int32(2) -04)------TableScan: t_pushdown projection=[a, b], partial_filters=[t_pushdown.b = Int32(2)] +03)----TableScan: t_pushdown projection=[a, b], partial_filters=[t_pushdown.b = Int32(2)] physical_plan 01)CoalescePartitionsExec 02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] @@ -176,8 +172,7 @@ EXPLAIN select a from t where b > 2 ORDER BY a; logical_plan 01)Sort: t.a ASC NULLS LAST 02)--Projection: t.a -03)----Filter: t.b > Int32(2) -04)------TableScan: t projection=[a, b], partial_filters=[t.b > Int32(2)] +03)----TableScan: t projection=[a, b], partial_filters=[t.b > Int32(2)] physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] @@ -189,8 +184,7 @@ EXPLAIN select a from t_pushdown where b > 2 ORDER BY a; logical_plan 01)Sort: t_pushdown.a ASC NULLS LAST 02)--Projection: t_pushdown.a -03)----Filter: t_pushdown.b > Int32(2) -04)------TableScan: t_pushdown projection=[a, b], partial_filters=[t_pushdown.b > Int32(2)] +03)----TableScan: t_pushdown projection=[a, b], partial_filters=[t_pushdown.b > Int32(2)] physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] @@ -212,8 +206,7 @@ EXPLAIN select a from t where b = 2 ORDER BY b; logical_plan 01)Projection: t.a 02)--Sort: t.b ASC NULLS LAST -03)----Filter: t.b = Int32(2) -04)------TableScan: t projection=[a, b], partial_filters=[t.b = Int32(2)] +03)----TableScan: t projection=[a, b], partial_filters=[t.b = Int32(2)] physical_plan 01)CoalescePartitionsExec 02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] @@ -224,8 +217,7 @@ EXPLAIN select a from t_pushdown where b = 2 ORDER BY b; logical_plan 01)Projection: t_pushdown.a 02)--Sort: t_pushdown.b ASC NULLS LAST -03)----Filter: t_pushdown.b = Int32(2) -04)------TableScan: t_pushdown projection=[a, b], partial_filters=[t_pushdown.b = Int32(2)] +03)----TableScan: t_pushdown projection=[a, b], partial_filters=[t_pushdown.b = Int32(2)] physical_plan 01)CoalescePartitionsExec 02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] @@ -258,8 +250,7 @@ EXPLAIN select a from t where b > 2 ORDER BY a; logical_plan 01)Sort: t.a ASC NULLS LAST 02)--Projection: t.a -03)----Filter: t.b > Int32(2) -04)------TableScan: t projection=[a, b], partial_filters=[t.b > Int32(2)] +03)----TableScan: t projection=[a, b], partial_filters=[t.b > Int32(2)] physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] @@ -273,8 +264,7 @@ EXPLAIN select a from t_pushdown where b > 2 ORDER BY a; logical_plan 01)Sort: t_pushdown.a ASC NULLS LAST 02)--Projection: t_pushdown.a -03)----Filter: t_pushdown.b > Int32(2) -04)------TableScan: t_pushdown projection=[a, b], partial_filters=[t_pushdown.b > Int32(2)] +03)----TableScan: t_pushdown projection=[a, b], partial_filters=[t_pushdown.b > Int32(2)] physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] @@ -296,8 +286,7 @@ EXPLAIN select a from t where b = 2 ORDER BY b; logical_plan 01)Projection: t.a 02)--Sort: t.b ASC NULLS LAST -03)----Filter: t.b = Int32(2) -04)------TableScan: t projection=[a, b], partial_filters=[t.b = Int32(2)] +03)----TableScan: t projection=[a, b], partial_filters=[t.b = Int32(2)] physical_plan 01)CoalescePartitionsExec 02)--FilterExec: b@1 = 2, projection=[a@0] @@ -310,8 +299,7 @@ EXPLAIN select a from t_pushdown where b = 2 ORDER BY b; logical_plan 01)Projection: t_pushdown.a 02)--Sort: t_pushdown.b ASC NULLS LAST -03)----Filter: t_pushdown.b = Int32(2) -04)------TableScan: t_pushdown projection=[a, b], partial_filters=[t_pushdown.b = Int32(2)] +03)----TableScan: t_pushdown projection=[a, b], partial_filters=[t_pushdown.b = Int32(2)] physical_plan 01)CoalescePartitionsExec 02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] @@ -333,8 +321,7 @@ EXPLAIN select a from t where b > 2 ORDER BY a; logical_plan 01)Sort: t.a ASC NULLS LAST 02)--Projection: t.a -03)----Filter: t.b > Int32(2) -04)------TableScan: t projection=[a, b], partial_filters=[t.b > Int32(2)] +03)----TableScan: t projection=[a, b], partial_filters=[t.b > Int32(2)] physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] @@ -353,8 +340,7 @@ EXPLAIN select a from t_pushdown where b = 2 ORDER BY b; logical_plan 01)Projection: t_pushdown.a 02)--Sort: t_pushdown.b ASC NULLS LAST -03)----Filter: t_pushdown.b = Int32(2) -04)------TableScan: t_pushdown projection=[a, b], partial_filters=[t_pushdown.b = Int32(2)] +03)----TableScan: t_pushdown projection=[a, b], partial_filters=[t_pushdown.b = Int32(2)] physical_plan 01)CoalescePartitionsExec 02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] @@ -372,8 +358,7 @@ EXPLAIN select a from t_pushdown where b > 2 AND a IS NOT NULL order by a; logical_plan 01)Sort: t_pushdown.a ASC NULLS LAST 02)--Projection: t_pushdown.a -03)----Filter: t_pushdown.b > Int32(2) AND t_pushdown.a IS NOT NULL -04)------TableScan: t_pushdown projection=[a, b], partial_filters=[t_pushdown.b > Int32(2), t_pushdown.a IS NOT NULL] +03)----TableScan: t_pushdown projection=[a, b], partial_filters=[t_pushdown.b > Int32(2), t_pushdown.a IS NOT NULL] physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] @@ -391,8 +376,7 @@ EXPLAIN select b from t_pushdown where a = 'bar' order by b; logical_plan 01)Sort: t_pushdown.b ASC NULLS LAST 02)--Projection: t_pushdown.b -03)----Filter: t_pushdown.a = Utf8View("bar") -04)------TableScan: t_pushdown projection=[a, b], partial_filters=[t_pushdown.a = Utf8View("bar")] +03)----TableScan: t_pushdown projection=[b, a], partial_filters=[t_pushdown.a = Utf8View("bar")] physical_plan 01)SortPreservingMergeExec: [b@0 ASC NULLS LAST] 02)--SortExec: expr=[b@0 ASC NULLS LAST], preserve_partitioning=[true] @@ -406,8 +390,7 @@ EXPLAIN select a from t_pushdown where b > random(); ---- logical_plan 01)Projection: t_pushdown.a -02)--Filter: CAST(t_pushdown.b AS Float64) > random() -03)----TableScan: t_pushdown projection=[a, b] +02)--TableScan: t_pushdown projection=[a, b], partial_filters=[CAST(t_pushdown.b AS Float64) > random()] physical_plan 01)FilterExec: CAST(b@1 AS Float64) > random(), projection=[a@0] 02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a, b], file_type=parquet @@ -463,9 +446,7 @@ xyz c query TT EXPLAIN select * from t_pushdown where part != val ---- -logical_plan -01)Filter: t_pushdown.val != t_pushdown.part -02)--TableScan: t_pushdown projection=[val, part], partial_filters=[t_pushdown.val != t_pushdown.part] +logical_plan TableScan: t_pushdown projection=[val, part], partial_filters=[t_pushdown.val != t_pushdown.part] physical_plan DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=b/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=c/file.parquet]]}, projection=[val, part], file_type=parquet, predicate=val@0 != part@1 # If we reference only a partition column it gets evaluated during the listing phase @@ -479,9 +460,7 @@ physical_plan DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion query TT EXPLAIN select * from t_pushdown where val != 'c'; ---- -logical_plan -01)Filter: t_pushdown.val != Utf8View("c") -02)--TableScan: t_pushdown projection=[val, part], partial_filters=[t_pushdown.val != Utf8View("c")] +logical_plan TableScan: t_pushdown projection=[val, part], partial_filters=[t_pushdown.val != Utf8View("c")] physical_plan DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=b/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=c/file.parquet]]}, projection=[val, part], file_type=parquet, predicate=val@0 != c, pruning_predicate=val_null_count@2 != row_count@3 AND (val_min@0 != c OR c != val_max@1), required_guarantees=[val not in (c)] # If we have a mix of filters: @@ -491,18 +470,14 @@ physical_plan DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion query TT EXPLAIN select * from t_pushdown where val != 'd' AND val != 'c' AND part = 'a' AND part != val; ---- -logical_plan -01)Filter: t_pushdown.val != Utf8View("d") AND t_pushdown.val != Utf8View("c") AND t_pushdown.val != t_pushdown.part -02)--TableScan: t_pushdown projection=[val, part], full_filters=[t_pushdown.part = Utf8View("a")], partial_filters=[t_pushdown.val != Utf8View("d"), t_pushdown.val != Utf8View("c"), t_pushdown.val != t_pushdown.part] +logical_plan TableScan: t_pushdown projection=[val, part], full_filters=[t_pushdown.part = Utf8View("a")], partial_filters=[t_pushdown.val != Utf8View("d"), t_pushdown.val != Utf8View("c"), t_pushdown.val != t_pushdown.part] physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet]]}, projection=[val, part], file_type=parquet, predicate=val@0 != d AND val@0 != c AND val@0 != part@1, pruning_predicate=val_null_count@2 != row_count@3 AND (val_min@0 != d OR d != val_max@1) AND val_null_count@2 != row_count@3 AND (val_min@0 != c OR c != val_max@1), required_guarantees=[val not in (c, d)] # The order of filters should not matter query TT EXPLAIN select val, part from t_pushdown where part = 'a' AND part = val; ---- -logical_plan -01)Filter: t_pushdown.val = t_pushdown.part -02)--TableScan: t_pushdown projection=[val, part], full_filters=[t_pushdown.part = Utf8View("a")], partial_filters=[t_pushdown.val = t_pushdown.part] +logical_plan TableScan: t_pushdown projection=[val, part], full_filters=[t_pushdown.part = Utf8View("a")], partial_filters=[t_pushdown.val = t_pushdown.part] physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet]]}, projection=[val, part], file_type=parquet, predicate=val@0 = part@1 query TT @@ -513,9 +488,7 @@ a a query TT EXPLAIN select val, part from t_pushdown where part = val AND part = 'a'; ---- -logical_plan -01)Filter: t_pushdown.val = t_pushdown.part -02)--TableScan: t_pushdown projection=[val, part], full_filters=[t_pushdown.part = Utf8View("a")], partial_filters=[t_pushdown.val = t_pushdown.part] +logical_plan TableScan: t_pushdown projection=[val, part], full_filters=[t_pushdown.part = Utf8View("a")], partial_filters=[t_pushdown.val = t_pushdown.part] physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet]]}, projection=[val, part], file_type=parquet, predicate=val@0 = part@1 query TT @@ -599,8 +572,7 @@ EXPLAIN SELECT id, tags FROM array_test WHERE array_has(tags, 'rust') ORDER BY i ---- logical_plan 01)Sort: array_test.id ASC NULLS LAST -02)--Filter: array_has(array_test.tags, Utf8("rust")) -03)----TableScan: array_test projection=[id, tags], partial_filters=[array_has(array_test.tags, Utf8("rust"))] +02)--TableScan: array_test projection=[id, tags], partial_filters=[array_has(array_test.tags, Utf8("rust"))] physical_plan 01)SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] 02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/array_data/data.parquet]]}, projection=[id, tags], file_type=parquet, predicate=array_has(tags@1, rust) @@ -616,8 +588,7 @@ EXPLAIN SELECT id, tags FROM array_test WHERE array_has_all(tags, ['rust', 'perf ---- logical_plan 01)Sort: array_test.id ASC NULLS LAST -02)--Filter: array_has_all(array_test.tags, List([rust, performance])) -03)----TableScan: array_test projection=[id, tags], partial_filters=[array_has_all(array_test.tags, List([rust, performance]))] +02)--TableScan: array_test projection=[id, tags], partial_filters=[array_has_all(array_test.tags, List([rust, performance]))] physical_plan 01)SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] 02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/array_data/data.parquet]]}, projection=[id, tags], file_type=parquet, predicate=array_has_all(tags@1, [rust, performance]) @@ -633,8 +604,7 @@ EXPLAIN SELECT id, tags FROM array_test WHERE array_has_any(tags, ['python', 'go ---- logical_plan 01)Sort: array_test.id ASC NULLS LAST -02)--Filter: array_has_any(array_test.tags, List([python, go])) -03)----TableScan: array_test projection=[id, tags], partial_filters=[array_has_any(array_test.tags, List([python, go]))] +02)--TableScan: array_test projection=[id, tags], partial_filters=[array_has_any(array_test.tags, List([python, go]))] physical_plan 01)SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] 02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/array_data/data.parquet]]}, projection=[id, tags], file_type=parquet, predicate=array_has_any(tags@1, [python, go]) @@ -652,8 +622,7 @@ EXPLAIN SELECT id, tags FROM array_test WHERE array_has_all(tags, ['rust']) OR a ---- logical_plan 01)Sort: array_test.id ASC NULLS LAST -02)--Filter: array_has_all(array_test.tags, List([rust])) OR array_has_any(array_test.tags, List([python, go])) -03)----TableScan: array_test projection=[id, tags], partial_filters=[array_has_all(array_test.tags, List([rust])) OR array_has_any(array_test.tags, List([python, go]))] +02)--TableScan: array_test projection=[id, tags], partial_filters=[array_has_all(array_test.tags, List([rust])) OR array_has_any(array_test.tags, List([python, go]))] physical_plan 01)SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] 02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/array_data/data.parquet]]}, projection=[id, tags], file_type=parquet, predicate=array_has_all(tags@1, [rust]) OR array_has_any(tags@1, [python, go]) @@ -669,8 +638,7 @@ EXPLAIN SELECT id, tags FROM array_test WHERE id > 1 AND array_has(tags, 'rust') ---- logical_plan 01)Sort: array_test.id ASC NULLS LAST -02)--Filter: array_test.id > Int64(1) AND array_has(array_test.tags, Utf8("rust")) -03)----TableScan: array_test projection=[id, tags], partial_filters=[array_test.id > Int64(1), array_has(array_test.tags, Utf8("rust"))] +02)--TableScan: array_test projection=[id, tags], partial_filters=[array_test.id > Int64(1), array_has(array_test.tags, Utf8("rust"))] physical_plan 01)SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] 02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/array_data/data.parquet]]}, projection=[id, tags], file_type=parquet, predicate=id@0 > 1 AND array_has(tags@1, rust), pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] diff --git a/datafusion/sqllogictest/test_files/predicates.slt b/datafusion/sqllogictest/test_files/predicates.slt index 7d33814b8bdbf..1c8e373db772c 100644 --- a/datafusion/sqllogictest/test_files/predicates.slt +++ b/datafusion/sqllogictest/test_files/predicates.slt @@ -379,59 +379,45 @@ set datafusion.explain.logical_plan_only = true query TT EXPLAIN SELECT * FROM test WHERE column1 = 'foo' OR column1 = 'bar' OR column1 = 'fazzz' ---- -logical_plan -01)Filter: test.column1 = Utf8("foo") OR test.column1 = Utf8("bar") OR test.column1 = Utf8("fazzz") -02)--TableScan: test projection=[column1] +logical_plan TableScan: test projection=[column1], unsupported_filters=[test.column1 = Utf8("foo") OR test.column1 = Utf8("bar") OR test.column1 = Utf8("fazzz")] # Number of OR statements is greater than threshold query TT EXPLAIN SELECT * FROM test WHERE column1 = 'foo' OR column1 = 'bar' OR column1 = 'fazzz' OR column1 = 'barfoo' ---- -logical_plan -01)Filter: test.column1 IN ([Utf8("foo"), Utf8("bar"), Utf8("fazzz"), Utf8("barfoo")]) -02)--TableScan: test projection=[column1] +logical_plan TableScan: test projection=[column1], unsupported_filters=[test.column1 IN ([Utf8("foo"), Utf8("bar"), Utf8("fazzz"), Utf8("barfoo")])] # Complex OR statements query TT EXPLAIN SELECT * FROM test WHERE column1 = 'foo' OR column1 = 'bar' OR column1 = 'fazzz' OR column1 = 'barfoo' OR false OR column1 = 'foobar' ---- -logical_plan -01)Filter: test.column1 IN ([Utf8("foo"), Utf8("bar"), Utf8("fazzz"), Utf8("barfoo"), Utf8("foobar")]) -02)--TableScan: test projection=[column1] +logical_plan TableScan: test projection=[column1], unsupported_filters=[test.column1 IN ([Utf8("foo"), Utf8("bar"), Utf8("fazzz"), Utf8("barfoo"), Utf8("foobar")])] # Balanced OR structures query TT EXPLAIN SELECT * FROM test WHERE (column1 = 'foo' OR column1 = 'bar') OR (column1 = 'fazzz' OR column1 = 'barfoo') ---- -logical_plan -01)Filter: test.column1 IN ([Utf8("foo"), Utf8("bar"), Utf8("fazzz"), Utf8("barfoo")]) -02)--TableScan: test projection=[column1] +logical_plan TableScan: test projection=[column1], unsupported_filters=[test.column1 IN ([Utf8("foo"), Utf8("bar"), Utf8("fazzz"), Utf8("barfoo")])] # Right-deep OR structures query TT EXPLAIN SELECT * FROM test WHERE column1 = 'foo' OR (column1 = 'bar' OR (column1 = 'fazzz' OR column1 = 'barfoo')) ---- -logical_plan -01)Filter: test.column1 IN ([Utf8("foo"), Utf8("bar"), Utf8("fazzz"), Utf8("barfoo")]) -02)--TableScan: test projection=[column1] +logical_plan TableScan: test projection=[column1], unsupported_filters=[test.column1 IN ([Utf8("foo"), Utf8("bar"), Utf8("fazzz"), Utf8("barfoo")])] # Not simplifiable, mixed column query TT EXPLAIN SELECT * FROM aggregate_test_100 WHERE (c2 = 1 OR c3 = 100) OR (c2 = 2 OR c2 = 3 OR c2 = 4) ---- -logical_plan -01)Filter: aggregate_test_100.c2 = Int8(1) OR aggregate_test_100.c3 = Int16(100) OR aggregate_test_100.c2 = Int8(2) OR aggregate_test_100.c2 = Int8(3) OR aggregate_test_100.c2 = Int8(4) -02)--TableScan: aggregate_test_100 projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], partial_filters=[aggregate_test_100.c2 = Int8(1) OR aggregate_test_100.c3 = Int16(100) OR aggregate_test_100.c2 = Int8(2) OR aggregate_test_100.c2 = Int8(3) OR aggregate_test_100.c2 = Int8(4)] +logical_plan TableScan: aggregate_test_100 projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], partial_filters=[aggregate_test_100.c2 = Int8(1) OR aggregate_test_100.c3 = Int16(100) OR aggregate_test_100.c2 = Int8(2) OR aggregate_test_100.c2 = Int8(3) OR aggregate_test_100.c2 = Int8(4)] # Partially simplifiable, mixed column query TT EXPLAIN SELECT * FROM aggregate_test_100 WHERE (c2 = 1 OR c3 = 100) OR (c2 = 2 OR c2 = 3 OR c2 = 4 OR c2 = 5) ---- -logical_plan -01)Filter: aggregate_test_100.c2 = Int8(1) OR aggregate_test_100.c3 = Int16(100) OR aggregate_test_100.c2 IN ([Int8(2), Int8(3), Int8(4), Int8(5)]) -02)--TableScan: aggregate_test_100 projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], partial_filters=[aggregate_test_100.c2 = Int8(1) OR aggregate_test_100.c3 = Int16(100) OR aggregate_test_100.c2 IN ([Int8(2), Int8(3), Int8(4), Int8(5)])] +logical_plan TableScan: aggregate_test_100 projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], partial_filters=[aggregate_test_100.c2 = Int8(1) OR aggregate_test_100.c3 = Int16(100) OR aggregate_test_100.c2 IN ([Int8(2), Int8(3), Int8(4), Int8(5)])] statement ok set datafusion.explain.logical_plan_only = false @@ -663,10 +649,8 @@ OR logical_plan 01)Projection: lineitem.l_partkey 02)--Inner Join: lineitem.l_partkey = part.p_partkey Filter: part.p_brand = Utf8View("Brand#12") 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 = Utf8View("Brand#23") 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 = Utf8View("Brand#34") AND lineitem.l_quantity >= Decimal128(Some(2000),15,2) AND lineitem.l_quantity <= Decimal128(Some(3000),15,2) AND part.p_size <= Int32(15) -03)----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) -04)------TableScan: lineitem projection=[l_partkey, l_quantity], partial_filters=[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)] -05)----Filter: (part.p_brand = Utf8View("Brand#12") AND part.p_size <= Int32(5) OR part.p_brand = Utf8View("Brand#23") AND part.p_size <= Int32(10) OR part.p_brand = Utf8View("Brand#34") AND part.p_size <= Int32(15)) AND part.p_size >= Int32(1) -06)------TableScan: part projection=[p_partkey, p_brand, p_size], partial_filters=[part.p_size >= Int32(1), part.p_brand = Utf8View("Brand#12") AND part.p_size <= Int32(5) OR part.p_brand = Utf8View("Brand#23") AND part.p_size <= Int32(10) OR part.p_brand = Utf8View("Brand#34") AND part.p_size <= Int32(15)] +03)----TableScan: lineitem projection=[l_partkey, l_quantity], partial_filters=[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)] +04)----TableScan: part projection=[p_partkey, p_brand, p_size], partial_filters=[part.p_size >= Int32(1), part.p_brand = Utf8View("Brand#12") AND part.p_size <= Int32(5) OR part.p_brand = Utf8View("Brand#23") AND part.p_size <= Int32(10) OR part.p_brand = Utf8View("Brand#34") AND part.p_size <= Int32(15)] physical_plan 01)HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], filter=p_brand@1 = Brand#12 AND l_quantity@0 >= Some(100),15,2 AND l_quantity@0 <= Some(1100),15,2 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND l_quantity@0 >= Some(1000),15,2 AND l_quantity@0 <= Some(2000),15,2 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND l_quantity@0 >= Some(2000),15,2 AND l_quantity@0 <= Some(3000),15,2 AND p_size@2 <= 15, projection=[l_partkey@0] 02)--RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 @@ -674,7 +658,7 @@ physical_plan 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_quantity], file_type=csv, has_header=true 06)--RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -07)----FilterExec: (p_brand@1 = Brand#12 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_size@2 <= 15) AND p_size@2 >= 1 +07)----FilterExec: p_size@2 >= 1 AND (p_brand@1 = Brand#12 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_size@2 <= 15) 08)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand, p_size], file_type=csv, has_header=true @@ -750,9 +734,8 @@ logical_plan 05)--------Inner Join: lineitem.l_partkey = part.p_partkey 06)----------TableScan: lineitem projection=[l_partkey, l_extendedprice, l_discount] 07)----------Projection: part.p_partkey -08)------------Filter: part.p_brand = Utf8View("Brand#12") OR part.p_brand = Utf8View("Brand#23") -09)--------------TableScan: part projection=[p_partkey, p_brand], partial_filters=[part.p_brand = Utf8View("Brand#12") OR part.p_brand = Utf8View("Brand#23")] -10)------TableScan: partsupp projection=[ps_partkey, ps_suppkey] +08)------------TableScan: part projection=[p_partkey, p_brand], partial_filters=[part.p_brand = Utf8View("Brand#12") OR part.p_brand = Utf8View("Brand#23")] +09)------TableScan: partsupp projection=[ps_partkey, ps_suppkey] physical_plan 01)AggregateExec: mode=SinglePartitioned, gby=[p_partkey@2 as p_partkey], aggr=[sum(lineitem.l_extendedprice), avg(lineitem.l_discount), count(DISTINCT partsupp.ps_suppkey)] 02)--ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, p_partkey@3 as p_partkey, ps_suppkey@0 as ps_suppkey] @@ -790,9 +773,7 @@ physical_plan EmptyExec query TT EXPLAIN FORMAT INDENT SELECT * FROM t WHERE x < 5 AND (10 * NULL < x); ---- -logical_plan -01)Filter: t.x < Int32(5) AND Boolean(NULL) -02)--TableScan: t projection=[x] +logical_plan TableScan: t projection=[x], unsupported_filters=[t.x < Int32(5), Boolean(NULL)] physical_plan 01)FilterExec: x@0 < 5 AND NULL 02)--DataSourceExec: partitions=1, partition_sizes=[1] @@ -800,9 +781,7 @@ physical_plan query TT EXPLAIN FORMAT INDENT SELECT * FROM t WHERE x < 5 OR (10 * NULL < x); ---- -logical_plan -01)Filter: t.x < Int32(5) OR Boolean(NULL) -02)--TableScan: t projection=[x] +logical_plan TableScan: t projection=[x], unsupported_filters=[t.x < Int32(5) OR Boolean(NULL)] physical_plan 01)FilterExec: x@0 < 5 OR NULL 02)--DataSourceExec: partitions=1, partition_sizes=[1] @@ -830,9 +809,7 @@ physical_plan DataSourceExec: partitions=1, partition_sizes=[1] query TT explain select x from t where x IN (1,2,3,4,5) AND x NOT IN (1,2,3,4); ---- -logical_plan -01)Filter: t.x = Int32(5) -02)--TableScan: t projection=[x] +logical_plan TableScan: t projection=[x], unsupported_filters=[t.x = Int32(5)] physical_plan 01)FilterExec: x@0 = 5 02)--DataSourceExec: partitions=1, partition_sizes=[1] diff --git a/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt b/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt index 34c5fd97b51f3..a6431830b8b39 100644 --- a/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt +++ b/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt @@ -325,8 +325,7 @@ logical_plan 06)----------SubqueryAlias: f 07)------------TableScan: fact_table_ordered projection=[value, f_dkey] 08)----------SubqueryAlias: d -09)------------Filter: dimension_table.service = Utf8View("log") -10)--------------TableScan: dimension_table projection=[d_dkey, env, service], partial_filters=[dimension_table.service = Utf8View("log")] +09)------------TableScan: dimension_table projection=[d_dkey, env, service], partial_filters=[dimension_table.service = Utf8View("log")] physical_plan 01)SortPreservingMergeExec: [f_dkey@0 ASC NULLS LAST] 02)--ProjectionExec: expr=[f_dkey@0 as f_dkey, max(d.env)@1 as max(d.env), max(d.service)@2 as max(d.service), count(Int64(1))@3 as count(*), sum(f.value)@4 as sum(f.value)] @@ -380,8 +379,7 @@ logical_plan 06)----------SubqueryAlias: f 07)------------TableScan: fact_table_ordered projection=[value, f_dkey] 08)----------SubqueryAlias: d -09)------------Filter: dimension_table.service = Utf8View("log") -10)--------------TableScan: dimension_table projection=[d_dkey, env, service], partial_filters=[dimension_table.service = Utf8View("log")] +09)------------TableScan: dimension_table projection=[d_dkey, env, service], partial_filters=[dimension_table.service = Utf8View("log")] physical_plan 01)SortPreservingMergeExec: [f_dkey@0 ASC NULLS LAST] 02)--ProjectionExec: expr=[f_dkey@0 as f_dkey, max(d.env)@1 as max(d.env), max(d.service)@2 as max(d.service), count(Int64(1))@3 as count(*), sum(f.value)@4 as sum(f.value)] diff --git a/datafusion/sqllogictest/test_files/projection.slt b/datafusion/sqllogictest/test_files/projection.slt index 5a4411233424a..818f8bfdaa263 100644 --- a/datafusion/sqllogictest/test_files/projection.slt +++ b/datafusion/sqllogictest/test_files/projection.slt @@ -272,8 +272,7 @@ select from t1 where t1.a > 1; ---- logical_plan 01)Projection: -02)--Filter: t1.a > Int64(1) -03)----TableScan: t1 projection=[a], partial_filters=[t1.a > Int64(1)] +02)--TableScan: t1 projection=[a], partial_filters=[t1.a > Int64(1)] physical_plan 01)ProjectionExec: expr=[] 02)--FilterExec: a@0 > 1 diff --git a/datafusion/sqllogictest/test_files/projection_pushdown.slt b/datafusion/sqllogictest/test_files/projection_pushdown.slt index 4be83589495e7..34cc964ac1088 100644 --- a/datafusion/sqllogictest/test_files/projection_pushdown.slt +++ b/datafusion/sqllogictest/test_files/projection_pushdown.slt @@ -218,8 +218,7 @@ EXPLAIN SELECT id, s['value'] FROM simple_struct WHERE id > 2; ---- logical_plan 01)Projection: simple_struct.id, get_field(simple_struct.s, Utf8("value")) -02)--Filter: simple_struct.id > Int64(2) -03)----TableScan: simple_struct projection=[id, s], partial_filters=[simple_struct.id > Int64(2)] +02)--TableScan: simple_struct projection=[id, s], partial_filters=[simple_struct.id > Int64(2)] physical_plan 01)ProjectionExec: expr=[id@0 as id, get_field(s@1, value) as simple_struct.s[value]] 02)--FilterExec: id@0 > 2 @@ -242,8 +241,7 @@ EXPLAIN SELECT id, s['value'] + 1 FROM simple_struct WHERE id > 2; ---- logical_plan 01)Projection: simple_struct.id, get_field(simple_struct.s, Utf8("value")) + Int64(1) -02)--Filter: simple_struct.id > Int64(2) -03)----TableScan: simple_struct projection=[id, s], partial_filters=[simple_struct.id > Int64(2)] +02)--TableScan: simple_struct projection=[id, s], partial_filters=[simple_struct.id > Int64(2)] physical_plan 01)ProjectionExec: expr=[id@0 as id, get_field(s@1, value) + 1 as simple_struct.s[value] + Int64(1)] 02)--FilterExec: id@0 > 2 @@ -266,8 +264,7 @@ EXPLAIN SELECT id, s['label'] FROM simple_struct WHERE s['value'] > 150; ---- logical_plan 01)Projection: simple_struct.id, get_field(simple_struct.s, Utf8("label")) -02)--Filter: get_field(simple_struct.s, Utf8("value")) > Int64(150) -03)----TableScan: simple_struct projection=[id, s], partial_filters=[get_field(simple_struct.s, Utf8("value")) > Int64(150)] +02)--TableScan: simple_struct projection=[id, s], partial_filters=[get_field(simple_struct.s, Utf8("value")) > Int64(150)] physical_plan 01)ProjectionExec: expr=[id@0 as id, get_field(s@1, label) as simple_struct.s[label]] 02)--FilterExec: get_field(s@1, value) > 150 @@ -539,8 +536,7 @@ EXPLAIN SELECT id, s['value'] FROM simple_struct WHERE id > 1 ORDER BY s['value' logical_plan 01)Sort: simple_struct.s[value] ASC NULLS LAST 02)--Projection: simple_struct.id, get_field(simple_struct.s, Utf8("value")) -03)----Filter: simple_struct.id > Int64(1) -04)------TableScan: simple_struct projection=[id, s], partial_filters=[simple_struct.id > Int64(1)] +03)----TableScan: simple_struct projection=[id, s], partial_filters=[simple_struct.id > Int64(1)] physical_plan 01)SortExec: expr=[simple_struct.s[value]@1 ASC NULLS LAST], preserve_partitioning=[false] 02)--ProjectionExec: expr=[id@0 as id, get_field(s@1, value) as simple_struct.s[value]] @@ -566,8 +562,7 @@ EXPLAIN SELECT id, s['value'] FROM simple_struct WHERE id > 1 ORDER BY s['value' logical_plan 01)Sort: simple_struct.s[value] ASC NULLS LAST, fetch=2 02)--Projection: simple_struct.id, get_field(simple_struct.s, Utf8("value")) -03)----Filter: simple_struct.id > Int64(1) -04)------TableScan: simple_struct projection=[id, s], partial_filters=[simple_struct.id > Int64(1)] +03)----TableScan: simple_struct projection=[id, s], partial_filters=[simple_struct.id > Int64(1)] physical_plan 01)SortExec: TopK(fetch=2), expr=[simple_struct.s[value]@1 ASC NULLS LAST], preserve_partitioning=[false] 02)--ProjectionExec: expr=[id@0 as id, get_field(s@1, value) as simple_struct.s[value]] @@ -591,8 +586,7 @@ EXPLAIN SELECT id, s['value'] + 1 FROM simple_struct WHERE id > 1 ORDER BY id LI logical_plan 01)Sort: simple_struct.id ASC NULLS LAST, fetch=2 02)--Projection: simple_struct.id, get_field(simple_struct.s, Utf8("value")) + Int64(1) -03)----Filter: simple_struct.id > Int64(1) -04)------TableScan: simple_struct projection=[id, s], partial_filters=[simple_struct.id > Int64(1)] +03)----TableScan: simple_struct projection=[id, s], partial_filters=[simple_struct.id > Int64(1)] physical_plan 01)SortExec: TopK(fetch=2), expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] 02)--ProjectionExec: expr=[id@0 as id, get_field(s@1, value) + 1 as simple_struct.s[value] + Int64(1)] @@ -730,8 +724,7 @@ EXPLAIN SELECT id, s['value'] FROM multi_struct WHERE id > 2 ORDER BY id; logical_plan 01)Sort: multi_struct.id ASC NULLS LAST 02)--Projection: multi_struct.id, get_field(multi_struct.s, Utf8("value")) -03)----Filter: multi_struct.id > Int64(2) -04)------TableScan: multi_struct projection=[id, s], partial_filters=[multi_struct.id > Int64(2)] +03)----TableScan: multi_struct projection=[id, s], partial_filters=[multi_struct.id > Int64(2)] physical_plan 01)SortPreservingMergeExec: [id@0 ASC NULLS LAST] 02)--SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[true] @@ -814,8 +807,7 @@ EXPLAIN SELECT id, s['label'] FROM nullable_struct WHERE s['value'] IS NOT NULL; ---- logical_plan 01)Projection: nullable_struct.id, get_field(nullable_struct.s, Utf8("label")) -02)--Filter: get_field(nullable_struct.s, Utf8("value")) IS NOT NULL -03)----TableScan: nullable_struct projection=[id, s], partial_filters=[get_field(nullable_struct.s, Utf8("value")) IS NOT NULL] +02)--TableScan: nullable_struct projection=[id, s], partial_filters=[get_field(nullable_struct.s, Utf8("value")) IS NOT NULL] physical_plan 01)ProjectionExec: expr=[id@0 as id, get_field(s@1, label) as nullable_struct.s[label]] 02)--FilterExec: get_field(s@1, value) IS NOT NULL @@ -938,13 +930,12 @@ EXPLAIN SELECT s['value'] + s['value'] as doubled FROM simple_struct WHERE id > logical_plan 01)Projection: __common_expr_1 + __common_expr_1 AS doubled 02)--Projection: get_field(simple_struct.s, Utf8("value")) AS __common_expr_1 -03)----Filter: simple_struct.id > Int64(2) -04)------TableScan: simple_struct projection=[id, s], partial_filters=[simple_struct.id > Int64(2)] +03)----TableScan: simple_struct projection=[s, id], partial_filters=[simple_struct.id > Int64(2)] physical_plan 01)ProjectionExec: expr=[__common_expr_1@0 + __common_expr_1@0 as doubled] 02)--ProjectionExec: expr=[get_field(s@0, value) as __common_expr_1] -03)----FilterExec: id@0 > 2, projection=[s@1] -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, s], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] +03)----FilterExec: id@1 > 2, projection=[s@0] +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[s, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] # Verify correctness query I @@ -963,12 +954,11 @@ EXPLAIN SELECT s['value'], s['label'] FROM simple_struct WHERE id > 2; ---- logical_plan 01)Projection: get_field(simple_struct.s, Utf8("value")), get_field(simple_struct.s, Utf8("label")) -02)--Filter: simple_struct.id > Int64(2) -03)----TableScan: simple_struct projection=[id, s], partial_filters=[simple_struct.id > Int64(2)] +02)--TableScan: simple_struct projection=[s, id], partial_filters=[simple_struct.id > Int64(2)] physical_plan 01)ProjectionExec: expr=[get_field(s@0, value) as simple_struct.s[value], get_field(s@0, label) as simple_struct.s[label]] -02)--FilterExec: id@0 > 2, projection=[s@1] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, s], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] +02)--FilterExec: id@1 > 2, projection=[s@0] +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[s, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] # Verify correctness query IT @@ -1012,12 +1002,11 @@ EXPLAIN SELECT s['value'] * 2 + length(s['label']) as score FROM simple_struct W ---- logical_plan 01)Projection: get_field(simple_struct.s, Utf8("value")) * Int64(2) + CAST(character_length(get_field(simple_struct.s, Utf8("label"))) AS length(get_field(simple_struct.s, Utf8("label"))) AS Int64) AS score -02)--Filter: simple_struct.id > Int64(1) -03)----TableScan: simple_struct projection=[id, s], partial_filters=[simple_struct.id > Int64(1)] +02)--TableScan: simple_struct projection=[s, id], partial_filters=[simple_struct.id > Int64(1)] physical_plan 01)ProjectionExec: expr=[get_field(s@0, value) * 2 + CAST(character_length(get_field(s@0, label)) AS Int64) as score] -02)--FilterExec: id@0 > 1, projection=[s@1] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, s], file_type=parquet, predicate=id@0 > 1, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] +02)--FilterExec: id@1 > 1, projection=[s@0] +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[s, id], file_type=parquet, predicate=id@0 > 1, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] # Verify correctness query I @@ -1088,8 +1077,7 @@ EXPLAIN SELECT id, s['value'] FROM simple_struct WHERE id > 1; ---- logical_plan 01)Projection: simple_struct.id, get_field(simple_struct.s, Utf8("value")) -02)--Filter: simple_struct.id > Int64(1) -03)----TableScan: simple_struct projection=[id, s], partial_filters=[simple_struct.id > Int64(1)] +02)--TableScan: simple_struct projection=[id, s], partial_filters=[simple_struct.id > Int64(1)] physical_plan 01)ProjectionExec: expr=[id@0 as id, get_field(s@1, value) as simple_struct.s[value]] 02)--FilterExec: id@0 > 1 @@ -1107,12 +1095,11 @@ EXPLAIN SELECT s['value'] FROM simple_struct WHERE id > 1 AND (id < 4 OR id = 5) ---- logical_plan 01)Projection: get_field(simple_struct.s, Utf8("value")) -02)--Filter: simple_struct.id > Int64(1) AND (simple_struct.id < Int64(4) OR simple_struct.id = Int64(5)) -03)----TableScan: simple_struct projection=[id, s], partial_filters=[simple_struct.id > Int64(1), simple_struct.id < Int64(4) OR simple_struct.id = Int64(5)] +02)--TableScan: simple_struct projection=[s, id], partial_filters=[simple_struct.id > Int64(1), simple_struct.id < Int64(4) OR simple_struct.id = Int64(5)] physical_plan 01)ProjectionExec: expr=[get_field(s@0, value) as simple_struct.s[value]] -02)--FilterExec: id@0 > 1 AND (id@0 < 4 OR id@0 = 5), projection=[s@1] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, s], file_type=parquet, predicate=id@0 > 1 AND (id@0 < 4 OR id@0 = 5), pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1 AND (id_null_count@1 != row_count@2 AND id_min@3 < 4 OR id_null_count@1 != row_count@2 AND id_min@3 <= 5 AND 5 <= id_max@0), required_guarantees=[] +02)--FilterExec: id@1 > 1 AND (id@1 < 4 OR id@1 = 5), projection=[s@0] +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[s, id], file_type=parquet, predicate=id@0 > 1 AND (id@0 < 4 OR id@0 = 5), pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1 AND (id_null_count@1 != row_count@2 AND id_min@3 < 4 OR id_null_count@1 != row_count@2 AND id_min@3 <= 5 AND 5 <= id_max@0), required_guarantees=[] # Verify correctness - should return rows where (id > 1) AND ((id < 4) OR (id = 5)) # That's: id=2,3 (1 1 AND id < 5; ---- logical_plan 01)Projection: get_field(simple_struct.s, Utf8("value")) -02)--Filter: simple_struct.id > Int64(1) AND simple_struct.id < Int64(5) -03)----TableScan: simple_struct projection=[id, s], partial_filters=[simple_struct.id > Int64(1), simple_struct.id < Int64(5)] +02)--TableScan: simple_struct projection=[s, id], partial_filters=[simple_struct.id > Int64(1), simple_struct.id < Int64(5)] physical_plan 01)ProjectionExec: expr=[get_field(s@0, value) as simple_struct.s[value]] -02)--FilterExec: id@0 > 1 AND id@0 < 5, projection=[s@1] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, s], file_type=parquet, predicate=id@0 > 1 AND id@0 < 5, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1 AND id_null_count@1 != row_count@2 AND id_min@3 < 5, required_guarantees=[] +02)--FilterExec: id@1 > 1 AND id@1 < 5, projection=[s@0] +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[s, id], file_type=parquet, predicate=id@0 > 1 AND id@0 < 5, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1 AND id_null_count@1 != row_count@2 AND id_min@3 < 5, required_guarantees=[] # Verify correctness - should return rows where 1 < id < 5 (id=2,3,4) query I @@ -1148,8 +1134,7 @@ EXPLAIN SELECT s['value'], s['label'], id FROM simple_struct WHERE id > 1; ---- logical_plan 01)Projection: get_field(simple_struct.s, Utf8("value")), get_field(simple_struct.s, Utf8("label")), simple_struct.id -02)--Filter: simple_struct.id > Int64(1) -03)----TableScan: simple_struct projection=[id, s], partial_filters=[simple_struct.id > Int64(1)] +02)--TableScan: simple_struct projection=[id, s], partial_filters=[simple_struct.id > Int64(1)] physical_plan 01)ProjectionExec: expr=[get_field(s@1, value) as simple_struct.s[value], get_field(s@1, label) as simple_struct.s[label], id@0 as id] 02)--FilterExec: id@0 > 1 @@ -1168,8 +1153,7 @@ EXPLAIN SELECT s['value'] FROM simple_struct WHERE length(s['label']) > 4; ---- logical_plan 01)Projection: get_field(simple_struct.s, Utf8("value")) -02)--Filter: character_length(get_field(simple_struct.s, Utf8("label"))) > Int32(4) -03)----TableScan: simple_struct projection=[s], partial_filters=[character_length(get_field(simple_struct.s, Utf8("label"))) > Int32(4)] +02)--TableScan: simple_struct projection=[s], partial_filters=[character_length(get_field(simple_struct.s, Utf8("label"))) > Int32(4)] physical_plan 01)ProjectionExec: expr=[get_field(s@0, value) as simple_struct.s[value]] 02)--FilterExec: character_length(get_field(s@0, label)) > 4 diff --git a/datafusion/sqllogictest/test_files/push_down_filter.slt b/datafusion/sqllogictest/test_files/push_down_filter.slt index b1cb354e053e4..49cd7303fdbef 100644 --- a/datafusion/sqllogictest/test_files/push_down_filter.slt +++ b/datafusion/sqllogictest/test_files/push_down_filter.slt @@ -40,7 +40,7 @@ physical_plan 02)--UnnestExec 03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 04)------ProjectionExec: expr=[column2@0 as __unnest_placeholder(v.column2)] -05)--------FilterExec: column1@0 = 2, projection=[column2@1] +05)--------FilterExec: column1@1 = 2, projection=[column2@0] 06)----------DataSourceExec: partitions=1, partition_sizes=[1] query I @@ -536,10 +536,8 @@ explain select * from t1 left join t2 on t1.k = t2.k where t1.k > 1; ---- logical_plan 01)Left Join: t1.k = t2.k -02)--Filter: t1.k > Int32(1) -03)----TableScan: t1 projection=[k, v] -04)--Filter: t2.k > Int32(1) -05)----TableScan: t2 projection=[k, v] +02)--TableScan: t1 projection=[k, v], unsupported_filters=[t1.k > Int32(1)] +03)--TableScan: t2 projection=[k, v], unsupported_filters=[t2.k > Int32(1)] query IIII rowsort select * from t1 left join t2 on t1.k = t2.k where t1.k > 1; @@ -555,9 +553,8 @@ explain select * from t1 left join t2 on t1.k = t2.k where t1.v > 1; ---- logical_plan 01)Left Join: t1.k = t2.k -02)--Filter: t1.v > Int32(1) -03)----TableScan: t1 projection=[k, v] -04)--TableScan: t2 projection=[k, v] +02)--TableScan: t1 projection=[k, v], unsupported_filters=[t1.v > Int32(1)] +03)--TableScan: t2 projection=[k, v] query IIII rowsort select * from t1 left join t2 on t1.k = t2.k where t1.v > 1; @@ -574,9 +571,8 @@ explain select * from t1 left join t2 on t1.k = t2.k where t1.k > 3 or t1.v > 20 ---- logical_plan 01)Left Join: t1.k = t2.k -02)--Filter: t1.k > Int32(3) OR t1.v > Int32(20) -03)----TableScan: t1 projection=[k, v] -04)--TableScan: t2 projection=[k, v] +02)--TableScan: t1 projection=[k, v], unsupported_filters=[t1.k > Int32(3) OR t1.v > Int32(20)] +03)--TableScan: t2 projection=[k, v] query IIII rowsort select * from t1 left join t2 on t1.k = t2.k where t1.k > 3 or t1.v > 20; @@ -592,10 +588,8 @@ explain select * from t1 right join t2 on t1.k = t2.k where t1.k > 1; ---- logical_plan 01)Inner Join: t1.k = t2.k -02)--Filter: t1.k > Int32(1) -03)----TableScan: t1 projection=[k, v] -04)--Filter: t2.k > Int32(1) -05)----TableScan: t2 projection=[k, v] +02)--TableScan: t1 projection=[k, v], unsupported_filters=[t1.k > Int32(1)] +03)--TableScan: t2 projection=[k, v], unsupported_filters=[t2.k > Int32(1)] query IIII rowsort select * from t1 right join t2 on t1.k = t2.k where t1.k > 1; @@ -609,9 +603,8 @@ explain select * from t1 right join t2 on t1.k = t2.k where t1.v > 1; ---- logical_plan 01)Inner Join: t1.k = t2.k -02)--Filter: t1.v > Int32(1) -03)----TableScan: t1 projection=[k, v] -04)--TableScan: t2 projection=[k, v] +02)--TableScan: t1 projection=[k, v], unsupported_filters=[t1.v > Int32(1)] +03)--TableScan: t2 projection=[k, v] query IIII rowsort select * from t1 right join t2 on t1.k = t2.k where t1.v > 1; @@ -626,9 +619,8 @@ explain select * from t1 right join t2 on t1.k = t2.k where t1.k > 3 or t1.v > 2 ---- logical_plan 01)Inner Join: t1.k = t2.k -02)--Filter: t1.k > Int32(3) OR t1.v > Int32(20) -03)----TableScan: t1 projection=[k, v] -04)--TableScan: t2 projection=[k, v] +02)--TableScan: t1 projection=[k, v], unsupported_filters=[t1.k > Int32(3) OR t1.v > Int32(20)] +03)--TableScan: t2 projection=[k, v] query IIII rowsort select * from t1 right join t2 on t1.k = t2.k where t1.k > 3 or t1.v > 20; @@ -641,10 +633,8 @@ explain select * from t1 left anti join t2 on t1.k = t2.k where t1.k > 1; ---- logical_plan 01)LeftAnti Join: t1.k = t2.k -02)--Filter: t1.k > Int32(1) -03)----TableScan: t1 projection=[k, v] -04)--Filter: t2.k > Int32(1) -05)----TableScan: t2 projection=[k] +02)--TableScan: t1 projection=[k, v], unsupported_filters=[t1.k > Int32(1)] +03)--TableScan: t2 projection=[k], unsupported_filters=[t2.k > Int32(1)] query II rowsort select * from t1 left anti join t2 on t1.k = t2.k where t1.k > 1; @@ -658,9 +648,8 @@ explain select * from t1 left anti join t2 on t1.k = t2.k where t1.v > 1; ---- logical_plan 01)LeftAnti Join: t1.k = t2.k -02)--Filter: t1.v > Int32(1) -03)----TableScan: t1 projection=[k, v] -04)--TableScan: t2 projection=[k] +02)--TableScan: t1 projection=[k, v], unsupported_filters=[t1.v > Int32(1)] +03)--TableScan: t2 projection=[k] query II rowsort select * from t1 left anti join t2 on t1.k = t2.k where t1.v > 1; @@ -674,9 +663,8 @@ explain select * from t1 left anti join t2 on t1.k = t2.k where t1.k > 3 or t1.v ---- logical_plan 01)LeftAnti Join: t1.k = t2.k -02)--Filter: t1.k > Int32(3) OR t1.v > Int32(20) -03)----TableScan: t1 projection=[k, v] -04)--TableScan: t2 projection=[k] +02)--TableScan: t1 projection=[k, v], unsupported_filters=[t1.k > Int32(3) OR t1.v > Int32(20)] +03)--TableScan: t2 projection=[k] query II rowsort select * from t1 left anti join t2 on t1.k = t2.k where t1.k > 3 or t1.v > 20; @@ -692,10 +680,8 @@ explain select * from t1 right anti join t2 on t1.k = t2.k where t2.k > 1; ---- logical_plan 01)RightAnti Join: t1.k = t2.k -02)--Filter: t1.k > Int32(1) -03)----TableScan: t1 projection=[k] -04)--Filter: t2.k > Int32(1) -05)----TableScan: t2 projection=[k, v] +02)--TableScan: t1 projection=[k], unsupported_filters=[t1.k > Int32(1)] +03)--TableScan: t2 projection=[k, v], unsupported_filters=[t2.k > Int32(1)] query II rowsort select * from t1 right anti join t2 on t1.k = t2.k where t2.k > 1; @@ -709,8 +695,7 @@ explain select * from t1 right anti join t2 on t1.k = t2.k where t2.v > 1; logical_plan 01)RightAnti Join: t1.k = t2.k 02)--TableScan: t1 projection=[k] -03)--Filter: t2.v > Int32(1) -04)----TableScan: t2 projection=[k, v] +03)--TableScan: t2 projection=[k, v], unsupported_filters=[t2.v > Int32(1)] query II rowsort select * from t1 right anti join t2 on t1.k = t2.k where t2.v > 1; @@ -724,8 +709,7 @@ explain select * from t1 right anti join t2 on t1.k = t2.k where t2.k > 3 or t2. logical_plan 01)RightAnti Join: t1.k = t2.k 02)--TableScan: t1 projection=[k] -03)--Filter: t2.k > Int32(3) OR t2.v > Int32(20) -04)----TableScan: t2 projection=[k, v] +03)--TableScan: t2 projection=[k, v], unsupported_filters=[t2.k > Int32(3) OR t2.v > Int32(20)] query II rowsort select * from t1 right anti join t2 on t1.k = t2.k where t2.k > 3 or t2.v > 20; diff --git a/datafusion/sqllogictest/test_files/pwmj.slt b/datafusion/sqllogictest/test_files/pwmj.slt index 295eb94318ee5..1167b701a9f8f 100644 --- a/datafusion/sqllogictest/test_files/pwmj.slt +++ b/datafusion/sqllogictest/test_files/pwmj.slt @@ -76,12 +76,10 @@ logical_plan 01)Sort: t1.t1_id ASC NULLS LAST 02)--Inner Join: Filter: t1.t1_id > t2.t2_id 03)----SubqueryAlias: t1 -04)------Filter: join_t1.t1_id > Int32(10) -05)--------TableScan: join_t1 projection=[t1_id] -06)----SubqueryAlias: t2 -07)------Projection: join_t2.t2_id -08)--------Filter: join_t2.t2_int > Int32(1) -09)----------TableScan: join_t2 projection=[t2_id, t2_int] +04)------TableScan: join_t1 projection=[t1_id], unsupported_filters=[join_t1.t1_id > Int32(10)] +05)----SubqueryAlias: t2 +06)------Projection: join_t2.t2_id +07)--------TableScan: join_t2 projection=[t2_id, t2_int], unsupported_filters=[join_t2.t2_int > Int32(1)] physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] 02)--SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] @@ -121,12 +119,10 @@ logical_plan 01)Sort: t1.t1_id ASC NULLS LAST, t2.t2_id ASC NULLS LAST 02)--Inner Join: Filter: t1.t1_id >= t2.t2_id 03)----SubqueryAlias: t1 -04)------Filter: join_t1.t1_id >= Int32(22) -05)--------TableScan: join_t1 projection=[t1_id] -06)----SubqueryAlias: t2 -07)------Projection: join_t2.t2_id -08)--------Filter: join_t2.t2_int = Int32(3) -09)----------TableScan: join_t2 projection=[t2_id, t2_int] +04)------TableScan: join_t1 projection=[t1_id], unsupported_filters=[join_t1.t1_id >= Int32(22)] +05)----SubqueryAlias: t2 +06)------Projection: join_t2.t2_id +07)--------TableScan: join_t2 projection=[t2_id, t2_int], unsupported_filters=[join_t2.t2_int = Int32(3)] physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST, t2_id@1 ASC NULLS LAST] 02)--SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] @@ -170,8 +166,7 @@ logical_plan 04)------TableScan: join_t1 projection=[t1_id] 05)----SubqueryAlias: t2 06)------Projection: join_t2.t2_id -07)--------Filter: join_t2.t2_int >= Int32(3) -08)----------TableScan: join_t2 projection=[t2_id, t2_int] +07)--------TableScan: join_t2 projection=[t2_id, t2_int], unsupported_filters=[join_t2.t2_int >= Int32(3)] physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST, t2_id@1 ASC NULLS LAST] 02)--SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] @@ -217,8 +212,7 @@ logical_plan 04)------TableScan: join_t1 projection=[t1_id] 05)----SubqueryAlias: t2 06)------Projection: join_t2.t2_id -07)--------Filter: join_t2.t2_int >= Int32(3) -08)----------TableScan: join_t2 projection=[t2_id, t2_int] +07)--------TableScan: join_t2 projection=[t2_id, t2_int], unsupported_filters=[join_t2.t2_int >= Int32(3)] physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST, t2_id@1 ASC NULLS LAST] 02)--SortExec: expr=[t1_id@0 ASC NULLS LAST, t2_id@1 ASC NULLS LAST], preserve_partitioning=[true] @@ -258,12 +252,10 @@ logical_plan 01)Sort: t1.t1_id ASC NULLS LAST, t2.t2_id ASC NULLS LAST 02)--Inner Join: Filter: t1.t1_id <= t2.t2_id 03)----SubqueryAlias: t1 -04)------Filter: join_t1.t1_id = Int32(11) OR join_t1.t1_id = Int32(44) -05)--------TableScan: join_t1 projection=[t1_id] -06)----SubqueryAlias: t2 -07)------Projection: join_t2.t2_id -08)--------Filter: join_t2.t2_name != Utf8View("y") -09)----------TableScan: join_t2 projection=[t2_id, t2_name] +04)------TableScan: join_t1 projection=[t1_id], unsupported_filters=[join_t1.t1_id = Int32(11) OR join_t1.t1_id = Int32(44)] +05)----SubqueryAlias: t2 +06)------Projection: join_t2.t2_id +07)--------TableScan: join_t2 projection=[t2_id, t2_name], unsupported_filters=[join_t2.t2_name != Utf8View("y")] physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST, t2_id@1 ASC NULLS LAST] 02)--SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] diff --git a/datafusion/sqllogictest/test_files/qualify.slt b/datafusion/sqllogictest/test_files/qualify.slt index ce58e3998cf57..0c9c634a2f332 100644 --- a/datafusion/sqllogictest/test_files/qualify.slt +++ b/datafusion/sqllogictest/test_files/qualify.slt @@ -312,8 +312,7 @@ logical_plan 05)--------Projection: users.dept, users.salary 06)----------Filter: sum(users.salary) > Decimal128(Some(2000000),20,2) 07)------------Aggregate: groupBy=[[users.dept, users.salary]], aggr=[[sum(users.salary)]] -08)--------------Filter: users.salary > Decimal128(Some(500000),10,2) -09)----------------TableScan: users projection=[salary, dept] +08)--------------TableScan: users projection=[salary, dept], unsupported_filters=[users.salary > Decimal128(Some(500000),10,2)] physical_plan 01)ProjectionExec: expr=[dept@0 as dept, avg(users.salary) PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as r] 02)--FilterExec: avg(users.salary) PARTITION BY [users.dept] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 > Some(60000000000),14,6 diff --git a/datafusion/sqllogictest/test_files/regexp/regexp_like.slt b/datafusion/sqllogictest/test_files/regexp/regexp_like.slt index 6f2d5a873c1b6..ec0af76ac4b02 100644 --- a/datafusion/sqllogictest/test_files/regexp/regexp_like.slt +++ b/datafusion/sqllogictest/test_files/regexp/regexp_like.slt @@ -247,9 +247,7 @@ Baz query TT explain select column1 from dict_table where column1 LIKE '%oo%'; ---- -logical_plan -01)Filter: dict_table.column1 LIKE Utf8("%oo%") -02)--TableScan: dict_table projection=[column1] +logical_plan TableScan: dict_table projection=[column1], unsupported_filters=[dict_table.column1 LIKE Utf8("%oo%")] physical_plan 01)FilterExec: column1@0 LIKE %oo% 02)--DataSourceExec: partitions=1, partition_sizes=[1] diff --git a/datafusion/sqllogictest/test_files/repartition.slt b/datafusion/sqllogictest/test_files/repartition.slt index 54e445f212422..a19d229aba771 100644 --- a/datafusion/sqllogictest/test_files/repartition.slt +++ b/datafusion/sqllogictest/test_files/repartition.slt @@ -115,8 +115,7 @@ EXPLAIN SELECT c1, c2, c3 FROM sink_table WHERE c3 > 0 LIMIT 5; ---- logical_plan 01)Limit: skip=0, fetch=5 -02)--Filter: sink_table.c3 > Int16(0) -03)----TableScan: sink_table projection=[c1, c2, c3] +02)--TableScan: sink_table projection=[c1, c2, c3], unsupported_filters=[sink_table.c3 > Int16(0)], fetch=5 physical_plan 01)CoalescePartitionsExec: fetch=5 02)--FilterExec: c3@2 > 0, fetch=5 diff --git a/datafusion/sqllogictest/test_files/repartition_scan.slt b/datafusion/sqllogictest/test_files/repartition_scan.slt index c9c2f91257081..ea1c9c2437475 100644 --- a/datafusion/sqllogictest/test_files/repartition_scan.slt +++ b/datafusion/sqllogictest/test_files/repartition_scan.slt @@ -59,9 +59,7 @@ select * from parquet_table; query TT EXPLAIN SELECT column1 FROM parquet_table WHERE column1 <> 42; ---- -logical_plan -01)Filter: parquet_table.column1 != Int32(42) -02)--TableScan: parquet_table projection=[column1], partial_filters=[parquet_table.column1 != Int32(42)] +logical_plan TableScan: parquet_table projection=[column1], partial_filters=[parquet_table.column1 != Int32(42)] physical_plan 01)FilterExec: column1@0 != 42 02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..135], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:135..270], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:270..405], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:405..537]]}, projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] @@ -74,9 +72,7 @@ set datafusion.optimizer.enable_round_robin_repartition = false; query TT EXPLAIN SELECT column1 FROM parquet_table WHERE column1 <> 42; ---- -logical_plan -01)Filter: parquet_table.column1 != Int32(42) -02)--TableScan: parquet_table projection=[column1], partial_filters=[parquet_table.column1 != Int32(42)] +logical_plan TableScan: parquet_table projection=[column1], partial_filters=[parquet_table.column1 != Int32(42)] physical_plan 01)FilterExec: column1@0 != 42 02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..135], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:135..270], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:270..405], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:405..537]]}, projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] @@ -97,8 +93,7 @@ EXPLAIN SELECT column1 FROM parquet_table WHERE column1 <> 42 ORDER BY column1; ---- logical_plan 01)Sort: parquet_table.column1 ASC NULLS LAST -02)--Filter: parquet_table.column1 != Int32(42) -03)----TableScan: parquet_table projection=[column1], partial_filters=[parquet_table.column1 != Int32(42)] +02)--TableScan: parquet_table projection=[column1], partial_filters=[parquet_table.column1 != Int32(42)] physical_plan 01)SortPreservingMergeExec: [column1@0 ASC NULLS LAST] 02)--SortExec: expr=[column1@0 ASC NULLS LAST], preserve_partitioning=[true] @@ -133,8 +128,7 @@ EXPLAIN SELECT column1 FROM parquet_table_with_order WHERE column1 <> 42 ORDER B ---- logical_plan 01)Sort: parquet_table_with_order.column1 ASC NULLS LAST -02)--Filter: parquet_table_with_order.column1 != Int32(42) -03)----TableScan: parquet_table_with_order projection=[column1], partial_filters=[parquet_table_with_order.column1 != Int32(42)] +02)--TableScan: parquet_table_with_order projection=[column1], partial_filters=[parquet_table_with_order.column1 != Int32(42)] physical_plan 01)SortPreservingMergeExec: [column1@0 ASC NULLS LAST] 02)--FilterExec: column1@0 != 42 @@ -179,9 +173,7 @@ select * from csv_table ORDER BY column1; query TT EXPLAIN SELECT column1 FROM csv_table WHERE column1 <> 42; ---- -logical_plan -01)Filter: csv_table.column1 != Int32(42) -02)--TableScan: csv_table projection=[column1], partial_filters=[csv_table.column1 != Int32(42)] +logical_plan TableScan: csv_table projection=[column1], partial_filters=[csv_table.column1 != Int32(42)] physical_plan 01)FilterExec: column1@0 != 42 02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/csv_table/1.csv:0..5], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/csv_table/1.csv:5..10], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/csv_table/1.csv:10..15], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/csv_table/1.csv:15..18]]}, projection=[column1], file_type=csv, has_header=true @@ -221,9 +213,7 @@ select * from "json_table" ORDER BY column1; query TT EXPLAIN SELECT column1 FROM "json_table" WHERE column1 <> 42; ---- -logical_plan -01)Filter: json_table.column1 != Int32(42) -02)--TableScan: json_table projection=[column1], partial_filters=[json_table.column1 != Int32(42)] +logical_plan TableScan: json_table projection=[column1], partial_filters=[json_table.column1 != Int32(42)] physical_plan 01)FilterExec: column1@0 != 42 02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/json_table/1.json:0..18], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/json_table/1.json:18..36], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/json_table/1.json:36..54], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/json_table/1.json:54..70]]}, projection=[column1], file_type=json diff --git a/datafusion/sqllogictest/test_files/repartition_subset_satisfaction.slt b/datafusion/sqllogictest/test_files/repartition_subset_satisfaction.slt index e2c9fa4237939..5a17e6d2faba8 100644 --- a/datafusion/sqllogictest/test_files/repartition_subset_satisfaction.slt +++ b/datafusion/sqllogictest/test_files/repartition_subset_satisfaction.slt @@ -362,10 +362,9 @@ logical_plan 09)----------------Inner Join: d.d_dkey = f.f_dkey 10)------------------SubqueryAlias: d 11)--------------------Projection: dimension_table.env, dimension_table.d_dkey -12)----------------------Filter: dimension_table.service = Utf8View("log") -13)------------------------TableScan: dimension_table projection=[env, service, d_dkey], partial_filters=[dimension_table.service = Utf8View("log")] -14)------------------SubqueryAlias: f -15)--------------------TableScan: fact_table_ordered projection=[timestamp, value, f_dkey] +12)----------------------TableScan: dimension_table projection=[env, d_dkey, service], partial_filters=[dimension_table.service = Utf8View("log")] +13)------------------SubqueryAlias: f +14)--------------------TableScan: fact_table_ordered projection=[timestamp, value, f_dkey] physical_plan 01)SortPreservingMergeExec: [env@0 ASC NULLS LAST, time_bin@1 ASC NULLS LAST] 02)--SortExec: expr=[env@0 ASC NULLS LAST, time_bin@1 ASC NULLS LAST], preserve_partitioning=[true] @@ -381,8 +380,8 @@ physical_plan 12)----------------------ProjectionExec: expr=[f_dkey@3 as f_dkey, env@0 as env, timestamp@1 as timestamp, value@2 as value] 13)------------------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_dkey@1, f_dkey@2)], projection=[env@0, timestamp@2, value@3, f_dkey@4] 14)--------------------------CoalescePartitionsExec -15)----------------------------FilterExec: service@1 = log, projection=[env@0, d_dkey@2] -16)------------------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=A/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=D/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=C/data.parquet]]}, projection=[env, service, d_dkey], file_type=parquet, predicate=service@1 = log, pruning_predicate=service_null_count@2 != row_count@3 AND service_min@0 <= log AND log <= service_max@1, required_guarantees=[service in (log)] +15)----------------------------FilterExec: service@2 = log, projection=[env@0, d_dkey@1] +16)------------------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=A/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=D/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=C/data.parquet]]}, projection=[env, d_dkey, service], file_type=parquet, predicate=service@1 = log, pruning_predicate=service_null_count@2 != row_count@3 AND service_min@0 <= log AND log <= service_max@1, required_guarantees=[service in (log)] 17)--------------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=C/data.parquet]]}, projection=[timestamp, value, f_dkey], output_ordering=[f_dkey@2 ASC NULLS LAST, timestamp@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify results without subset satisfaction @@ -461,10 +460,9 @@ logical_plan 09)----------------Inner Join: d.d_dkey = f.f_dkey 10)------------------SubqueryAlias: d 11)--------------------Projection: dimension_table.env, dimension_table.d_dkey -12)----------------------Filter: dimension_table.service = Utf8View("log") -13)------------------------TableScan: dimension_table projection=[env, service, d_dkey], partial_filters=[dimension_table.service = Utf8View("log")] -14)------------------SubqueryAlias: f -15)--------------------TableScan: fact_table_ordered projection=[timestamp, value, f_dkey] +12)----------------------TableScan: dimension_table projection=[env, d_dkey, service], partial_filters=[dimension_table.service = Utf8View("log")] +13)------------------SubqueryAlias: f +14)--------------------TableScan: fact_table_ordered projection=[timestamp, value, f_dkey] physical_plan 01)SortPreservingMergeExec: [env@0 ASC NULLS LAST, time_bin@1 ASC NULLS LAST] 02)--SortExec: expr=[env@0 ASC NULLS LAST, time_bin@1 ASC NULLS LAST], preserve_partitioning=[true] @@ -477,8 +475,8 @@ physical_plan 09)----------------ProjectionExec: expr=[f_dkey@3 as f_dkey, env@0 as env, timestamp@1 as timestamp, value@2 as value] 10)------------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_dkey@1, f_dkey@2)], projection=[env@0, timestamp@2, value@3, f_dkey@4] 11)--------------------CoalescePartitionsExec -12)----------------------FilterExec: service@1 = log, projection=[env@0, d_dkey@2] -13)------------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=A/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=D/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=C/data.parquet]]}, projection=[env, service, d_dkey], file_type=parquet, predicate=service@1 = log, pruning_predicate=service_null_count@2 != row_count@3 AND service_min@0 <= log AND log <= service_max@1, required_guarantees=[service in (log)] +12)----------------------FilterExec: service@2 = log, projection=[env@0, d_dkey@1] +13)------------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=A/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=D/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=C/data.parquet]]}, projection=[env, d_dkey, service], file_type=parquet, predicate=service@1 = log, pruning_predicate=service_null_count@2 != row_count@3 AND service_min@0 <= log AND log <= service_max@1, required_guarantees=[service in (log)] 14)--------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=C/data.parquet]]}, projection=[timestamp, value, f_dkey], output_ordering=[f_dkey@2 ASC NULLS LAST, timestamp@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify results match with subset satisfaction diff --git a/datafusion/sqllogictest/test_files/select.slt b/datafusion/sqllogictest/test_files/select.slt index 490df4b72d17b..6123d3e30f399 100644 --- a/datafusion/sqllogictest/test_files/select.slt +++ b/datafusion/sqllogictest/test_files/select.slt @@ -1434,8 +1434,7 @@ ORDER BY b, c; ---- logical_plan 01)Sort: annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST -02)--Filter: annotated_data_finite2.a = Int32(0) -03)----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0)] +02)--TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0)] physical_plan 01)SortPreservingMergeExec: [b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] 02)--FilterExec: a@1 = 0 @@ -1454,8 +1453,7 @@ ORDER BY c; ---- logical_plan 01)Sort: annotated_data_finite2.c ASC NULLS LAST -02)--Filter: annotated_data_finite2.a = Int32(0) AND annotated_data_finite2.b = Int32(0) -03)----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0), annotated_data_finite2.b = Int32(0)] +02)--TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0), annotated_data_finite2.b = Int32(0)] physical_plan 01)SortPreservingMergeExec: [c@3 ASC NULLS LAST] 02)--FilterExec: a@1 = 0 AND b@2 = 0 @@ -1474,8 +1472,7 @@ ORDER BY b, c; ---- logical_plan 01)Sort: annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST -02)--Filter: annotated_data_finite2.a = Int32(0) AND annotated_data_finite2.b = Int32(0) -03)----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0), annotated_data_finite2.b = Int32(0)] +02)--TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0), annotated_data_finite2.b = Int32(0)] physical_plan 01)SortPreservingMergeExec: [b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] 02)--FilterExec: a@1 = 0 AND b@2 = 0 @@ -1494,8 +1491,7 @@ ORDER BY a, b, c; ---- logical_plan 01)Sort: annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST -02)--Filter: annotated_data_finite2.a = Int32(0) AND annotated_data_finite2.b = Int32(0) -03)----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0), annotated_data_finite2.b = Int32(0)] +02)--TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0), annotated_data_finite2.b = Int32(0)] physical_plan 01)SortPreservingMergeExec: [a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] 02)--FilterExec: a@1 = 0 AND b@2 = 0 @@ -1514,8 +1510,7 @@ ORDER BY c; ---- logical_plan 01)Sort: annotated_data_finite2.c ASC NULLS LAST -02)--Filter: annotated_data_finite2.a = Int32(0) OR annotated_data_finite2.b = Int32(0) -03)----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0) OR annotated_data_finite2.b = Int32(0)] +02)--TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0) OR annotated_data_finite2.b = Int32(0)] physical_plan 01)SortPreservingMergeExec: [c@3 ASC NULLS LAST] 02)--SortExec: expr=[c@3 ASC NULLS LAST], preserve_partitioning=[true] @@ -1561,8 +1556,7 @@ ORDER BY CAST(ROUND(b) as INT); ---- logical_plan 01)Sort: CAST(round(CAST(annotated_data_finite2.b AS Float64)) AS Int32) ASC NULLS LAST -02)--Filter: CAST(round(CAST(annotated_data_finite2.b AS Float64)) AS Int32) = annotated_data_finite2.a -03)----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[CAST(round(CAST(annotated_data_finite2.b AS Float64)) AS Int32) = annotated_data_finite2.a] +02)--TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[CAST(round(CAST(annotated_data_finite2.b AS Float64)) AS Int32) = annotated_data_finite2.a] physical_plan 01)SortPreservingMergeExec: [CAST(round(CAST(b@2 AS Float64)) AS Int32) ASC NULLS LAST] 02)--FilterExec: CAST(round(CAST(b@2 AS Float64)) AS Int32) = a@1 diff --git a/datafusion/sqllogictest/test_files/simplify_expr.slt b/datafusion/sqllogictest/test_files/simplify_expr.slt index 99fc9900ef619..84abebc7d5ef3 100644 --- a/datafusion/sqllogictest/test_files/simplify_expr.slt +++ b/datafusion/sqllogictest/test_files/simplify_expr.slt @@ -22,9 +22,7 @@ create table t(a int, b string) as values (1, 'a'), (2, NULL), (NULL, 'c'); query TT explain select a from t where a BETWEEN 3 and 3 ---- -logical_plan -01)Filter: t.a = Int32(3) -02)--TableScan: t projection=[a] +logical_plan TableScan: t projection=[a], unsupported_filters=[t.a = Int32(3)] physical_plan 01)FilterExec: a@0 = 3 02)--DataSourceExec: partitions=1, partition_sizes=[1] @@ -33,9 +31,7 @@ physical_plan query TT explain select b from t where b ~ '.*' ---- -logical_plan -01)Filter: t.b ~ Utf8View(".*") -02)--TableScan: t projection=[b] +logical_plan TableScan: t projection=[b], unsupported_filters=[t.b ~ Utf8View(".*")] physical_plan 01)FilterExec: b@0 ~ .* 02)--DataSourceExec: partitions=1, partition_sizes=[1] @@ -43,9 +39,7 @@ physical_plan query TT explain select b from t where b !~ '.*' ---- -logical_plan -01)Filter: t.b !~ Utf8View(".*") -02)--TableScan: t projection=[b] +logical_plan TableScan: t projection=[b], unsupported_filters=[t.b !~ Utf8View(".*")] physical_plan 01)FilterExec: b@0 !~ .* 02)--DataSourceExec: partitions=1, partition_sizes=[1] @@ -63,9 +57,7 @@ select b from t where b !~ '.*' query TT explain select * from t where a = a; ---- -logical_plan -01)Filter: t.a IS NOT NULL OR Boolean(NULL) -02)--TableScan: t projection=[a, b] +logical_plan TableScan: t projection=[a, b], unsupported_filters=[t.a IS NOT NULL OR Boolean(NULL)] physical_plan 01)FilterExec: a@0 IS NOT NULL OR NULL 02)--DataSourceExec: partitions=1, partition_sizes=[1] diff --git a/datafusion/sqllogictest/test_files/simplify_predicates.slt b/datafusion/sqllogictest/test_files/simplify_predicates.slt index c2a21ea7103c3..ccdb3bcce8d30 100644 --- a/datafusion/sqllogictest/test_files/simplify_predicates.slt +++ b/datafusion/sqllogictest/test_files/simplify_predicates.slt @@ -34,104 +34,80 @@ CREATE TABLE test_data ( query TT EXPLAIN SELECT * FROM test_data WHERE int_col > 5 AND int_col > 6; ---- -logical_plan -01)Filter: test_data.int_col > Int32(6) -02)--TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col] +logical_plan TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col], unsupported_filters=[test_data.int_col > Int32(6)] # x > 5 AND x >= 6 should simplify to x >= 6 query TT EXPLAIN SELECT * FROM test_data WHERE int_col > 5 AND int_col >= 6; ---- -logical_plan -01)Filter: test_data.int_col >= Int32(6) -02)--TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col] +logical_plan TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col], unsupported_filters=[test_data.int_col >= Int32(6)] # x < 10 AND x <= 8 should simplify to x <= 8 query TT EXPLAIN SELECT * FROM test_data WHERE int_col < 10 AND int_col <= 8; ---- -logical_plan -01)Filter: test_data.int_col <= Int32(8) -02)--TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col] +logical_plan TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col], unsupported_filters=[test_data.int_col <= Int32(8)] # x > 5 AND x > 6 AND x > 7 should simplify to x > 7 query TT EXPLAIN SELECT * FROM test_data WHERE int_col > 5 AND int_col > 6 AND int_col > 7; ---- -logical_plan -01)Filter: test_data.int_col > Int32(7) -02)--TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col] +logical_plan TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col], unsupported_filters=[test_data.int_col > Int32(7)] # x > 5 AND y < 10 AND x > 6 AND y < 8 should simplify to x > 6 AND y < 8 query TT EXPLAIN SELECT * FROM test_data WHERE int_col > 5 AND float_col < 10 AND int_col > 6 AND float_col < 8; ---- -logical_plan -01)Filter: test_data.float_col < Float32(8) AND test_data.int_col > Int32(6) -02)--TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col] +logical_plan TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col], unsupported_filters=[test_data.float_col < Float32(8), test_data.int_col > Int32(6)] # x = 7 AND x = 7 should simplify to x = 7 query TT EXPLAIN SELECT * FROM test_data WHERE int_col = 7 AND int_col = 7; ---- -logical_plan -01)Filter: test_data.int_col = Int32(7) -02)--TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col] +logical_plan TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col], unsupported_filters=[test_data.int_col = Int32(7)] # x = 7 AND x = 6 should simplify to false query TT EXPLAIN SELECT * FROM test_data WHERE int_col = 7 AND int_col = 6; ---- -logical_plan EmptyRelation: rows=0 +logical_plan TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col], unsupported_filters=[Boolean(false)] # TODO: x = 7 AND x < 2 should simplify to false query TT EXPLAIN SELECT * FROM test_data WHERE int_col = 7 AND int_col < 2; ---- -logical_plan -01)Filter: test_data.int_col = Int32(7) AND test_data.int_col < Int32(2) -02)--TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col] +logical_plan TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col], unsupported_filters=[test_data.int_col = Int32(7), test_data.int_col < Int32(2)] # TODO: x = 7 AND x > 5 should simplify to x = 7 query TT EXPLAIN SELECT * FROM test_data WHERE int_col = 7 AND int_col > 5; ---- -logical_plan -01)Filter: test_data.int_col = Int32(7) AND test_data.int_col > Int32(5) -02)--TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col] +logical_plan TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col], unsupported_filters=[test_data.int_col = Int32(7), test_data.int_col > Int32(5)] # str_col > 'apple' AND str_col > 'banana' should simplify to str_col > 'banana' query TT EXPLAIN SELECT * FROM test_data WHERE str_col > 'apple' AND str_col > 'banana'; ---- -logical_plan -01)Filter: test_data.str_col > Utf8View("banana") -02)--TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col] +logical_plan TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col], unsupported_filters=[test_data.str_col > Utf8View("banana")] # date_col > '2023-01-01' AND date_col > '2023-02-01' should simplify to date_col > '2023-02-01' query TT EXPLAIN SELECT * FROM test_data WHERE date_col > '2023-01-01' AND date_col > '2023-02-01'; ---- -logical_plan -01)Filter: test_data.date_col > Date32("2023-02-01") -02)--TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col] +logical_plan TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col], unsupported_filters=[test_data.date_col > Date32("2023-02-01")] query TT EXPLAIN SELECT * FROM test_data WHERE bool_col = true AND bool_col = false; ---- -logical_plan -01)Filter: test_data.bool_col AND NOT test_data.bool_col -02)--TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col] +logical_plan TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col], unsupported_filters=[test_data.bool_col, NOT test_data.bool_col] # This shouldn't be simplified since they're different relationships query TT EXPLAIN SELECT * FROM test_data WHERE int_col > float_col AND int_col > 5; ---- -logical_plan -01)Filter: CAST(test_data.int_col AS Float32) > test_data.float_col AND test_data.int_col > Int32(5) -02)--TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col] +logical_plan TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col], unsupported_filters=[CAST(test_data.int_col AS Float32) > test_data.float_col, test_data.int_col > Int32(5)] # Should simplify the int_col predicates but preserve the others query TT @@ -141,9 +117,7 @@ WHERE int_col > 5 AND str_col LIKE 'A%' AND float_col BETWEEN 1 AND 100; ---- -logical_plan -01)Filter: test_data.str_col LIKE Utf8View("A%") AND test_data.float_col >= Float32(1) AND test_data.float_col <= Float32(100) AND test_data.int_col > Int32(10) -02)--TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col] +logical_plan TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col], unsupported_filters=[test_data.str_col LIKE Utf8View("A%"), test_data.float_col >= Float32(1), test_data.float_col <= Float32(100), test_data.int_col > Int32(10)] statement ok CREATE TABLE test_data2 ( @@ -164,50 +138,40 @@ logical_plan 01)Projection: t1.int_col, t2.value 02)--Inner Join: t1.int_col = t2.id 03)----SubqueryAlias: t1 -04)------Filter: test_data.int_col > Int32(10) -05)--------TableScan: test_data projection=[int_col] -06)----SubqueryAlias: t2 -07)------Filter: test_data2.value < Int32(50) AND test_data2.id > Int32(10) -08)--------TableScan: test_data2 projection=[id, value] +04)------TableScan: test_data projection=[int_col], unsupported_filters=[test_data.int_col > Int32(10)] +05)----SubqueryAlias: t2 +06)------TableScan: test_data2 projection=[id, value], unsupported_filters=[test_data2.value < Int32(50), test_data2.id > Int32(10)] # Handling negated predicates # NOT (x < 10) AND NOT (x < 5) should simplify to NOT (x < 10) query TT EXPLAIN SELECT * FROM test_data WHERE NOT (int_col < 10) AND NOT (int_col < 5); ---- -logical_plan -01)Filter: test_data.int_col >= Int32(10) -02)--TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col] +logical_plan TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col], unsupported_filters=[test_data.int_col >= Int32(10)] # x > 5 AND x < 10 should be preserved (can't be simplified) query TT EXPLAIN SELECT * FROM test_data WHERE int_col > 5 AND int_col < 10; ---- -logical_plan -01)Filter: test_data.int_col > Int32(5) AND test_data.int_col < Int32(10) -02)--TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col] +logical_plan TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col], unsupported_filters=[test_data.int_col > Int32(5), test_data.int_col < Int32(10)] # 5 < x AND 3 < x should simplify to 5 < x query TT EXPLAIN SELECT * FROM test_data WHERE 5 < int_col AND 3 < int_col; ---- -logical_plan -01)Filter: test_data.int_col > Int32(5) -02)--TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col] +logical_plan TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col], unsupported_filters=[test_data.int_col > Int32(5)] # CAST(x AS FLOAT) > 5.0 AND CAST(x AS FLOAT) > 6.0 should simplify query TT EXPLAIN SELECT * FROM test_data WHERE CAST(int_col AS FLOAT) > 5.0 AND CAST(int_col AS FLOAT) > 6.0; ---- -logical_plan -01)Filter: CAST(CAST(test_data.int_col AS Float32) AS Float64) > Float64(6) -02)--TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col] +logical_plan TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col], unsupported_filters=[CAST(CAST(test_data.int_col AS Float32) AS Float64) > Float64(5), CAST(CAST(test_data.int_col AS Float32) AS Float64) > Float64(6)] # x = 5 AND x = 6 (logically impossible) query TT EXPLAIN SELECT * FROM test_data WHERE int_col = 5 AND int_col = 6; ---- -logical_plan EmptyRelation: rows=0 +logical_plan TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col], unsupported_filters=[Boolean(false)] # (x > 5 OR y < 10) AND (x > 6 OR y < 8) # This is more complex but could still benefit from some simplification @@ -216,9 +180,7 @@ EXPLAIN SELECT * FROM test_data WHERE (int_col > 5 OR float_col < 10) AND (int_col > 6 OR float_col < 8); ---- -logical_plan -01)Filter: (test_data.int_col > Int32(5) OR test_data.float_col < Float32(10)) AND (test_data.int_col > Int32(6) OR test_data.float_col < Float32(8)) -02)--TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col] +logical_plan TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col], unsupported_filters=[test_data.int_col > Int32(5) OR test_data.float_col < Float32(10), test_data.int_col > Int32(6) OR test_data.float_col < Float32(8)] # Combination of AND and OR with simplifiable predicates query TT @@ -226,9 +188,7 @@ EXPLAIN SELECT * FROM test_data WHERE (int_col > 5 AND int_col > 6) OR (float_col < 10 AND float_col < 8); ---- -logical_plan -01)Filter: test_data.int_col > Int32(5) AND test_data.int_col > Int32(6) OR test_data.float_col < Float32(10) AND test_data.float_col < Float32(8) -02)--TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col] +logical_plan TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col], unsupported_filters=[test_data.int_col > Int32(5) AND test_data.int_col > Int32(6) OR test_data.float_col < Float32(10) AND test_data.float_col < Float32(8)] query TT @@ -237,9 +197,7 @@ EXPLAIN SELECT * FROM ( WHERE int_col > 1 AND int_col < 10 ) WHERE int_col >= 1 AND int_col <= 10; ---- -logical_plan -01)Filter: test_data.int_col > Int32(1) AND test_data.int_col < Int32(10) -02)--TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col] +logical_plan TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col], unsupported_filters=[test_data.int_col > Int32(1), test_data.int_col < Int32(10)] statement ok diff --git a/datafusion/sqllogictest/test_files/sort_pushdown.slt b/datafusion/sqllogictest/test_files/sort_pushdown.slt index 58d9915a24be2..12b00cd352d8e 100644 --- a/datafusion/sqllogictest/test_files/sort_pushdown.slt +++ b/datafusion/sqllogictest/test_files/sort_pushdown.slt @@ -151,8 +151,7 @@ ORDER BY id DESC LIMIT 5; ---- logical_plan 01)Sort: multi_rg_sorted.id DESC NULLS FIRST, fetch=5 -02)--Filter: multi_rg_sorted.category = Utf8View("alpha") OR multi_rg_sorted.category = Utf8View("gamma") -03)----TableScan: multi_rg_sorted projection=[id, category, value], partial_filters=[multi_rg_sorted.category = Utf8View("alpha") OR multi_rg_sorted.category = Utf8View("gamma")] +02)--TableScan: multi_rg_sorted projection=[id, category, value], partial_filters=[multi_rg_sorted.category = Utf8View("alpha") OR multi_rg_sorted.category = Utf8View("gamma")] physical_plan 01)SortExec: TopK(fetch=5), expr=[id@0 DESC], preserve_partitioning=[false] 02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/multi_rg_sorted.parquet]]}, projection=[id, category, value], file_type=parquet, predicate=(category@1 = alpha OR category@1 = gamma) AND DynamicFilter [ empty ], reverse_row_groups=true, pruning_predicate=category_null_count@2 != row_count@3 AND category_min@0 <= alpha AND alpha <= category_max@1 OR category_null_count@2 != row_count@3 AND category_min@0 <= gamma AND gamma <= category_max@1, required_guarantees=[category in (alpha, gamma)] @@ -377,8 +376,7 @@ LIMIT 2; ---- logical_plan 01)Sort: timeseries_parquet.period_end DESC NULLS FIRST, fetch=2 -02)--Filter: timeseries_parquet.timeframe = Utf8View("quarterly") -03)----TableScan: timeseries_parquet projection=[timeframe, period_end, value], partial_filters=[timeseries_parquet.timeframe = Utf8View("quarterly")] +02)--TableScan: timeseries_parquet projection=[timeframe, period_end, value], partial_filters=[timeseries_parquet.timeframe = Utf8View("quarterly")] physical_plan 01)SortExec: TopK(fetch=2), expr=[period_end@1 DESC], preserve_partitioning=[false] 02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/timeseries_sorted.parquet]]}, projection=[timeframe, period_end, value], file_type=parquet, predicate=timeframe@0 = quarterly AND DynamicFilter [ empty ], reverse_row_groups=true, pruning_predicate=timeframe_null_count@2 != row_count@3 AND timeframe_min@0 <= quarterly AND quarterly <= timeframe_max@1, required_guarantees=[timeframe in (quarterly)] @@ -402,8 +400,7 @@ LIMIT 2; ---- logical_plan 01)Sort: timeseries_parquet.period_end ASC NULLS LAST, fetch=2 -02)--Filter: timeseries_parquet.timeframe = Utf8View("quarterly") -03)----TableScan: timeseries_parquet projection=[timeframe, period_end, value], partial_filters=[timeseries_parquet.timeframe = Utf8View("quarterly")] +02)--TableScan: timeseries_parquet projection=[timeframe, period_end, value], partial_filters=[timeseries_parquet.timeframe = Utf8View("quarterly")] physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/timeseries_sorted.parquet]]}, projection=[timeframe, period_end, value], limit=2, output_ordering=[timeframe@0 ASC NULLS LAST, period_end@1 ASC NULLS LAST], file_type=parquet, predicate=timeframe@0 = quarterly, pruning_predicate=timeframe_null_count@2 != row_count@3 AND timeframe_min@0 <= quarterly AND quarterly <= timeframe_max@1, required_guarantees=[timeframe in (quarterly)] # Test 2.4: Verify ASC results @@ -454,8 +451,7 @@ LIMIT 2; ---- logical_plan 01)Sort: timeseries_parquet.period_end DESC NULLS FIRST, fetch=2 -02)--Filter: timeseries_parquet.timeframe = Utf8View("quarterly") -03)----TableScan: timeseries_parquet projection=[timeframe, period_end, value], partial_filters=[timeseries_parquet.timeframe = Utf8View("quarterly")] +02)--TableScan: timeseries_parquet projection=[timeframe, period_end, value], partial_filters=[timeseries_parquet.timeframe = Utf8View("quarterly")] physical_plan 01)SortExec: TopK(fetch=2), expr=[period_end@1 DESC], preserve_partitioning=[false] 02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/timeseries_sorted.parquet]]}, projection=[timeframe, period_end, value], output_ordering=[timeframe@0 ASC NULLS LAST, period_end@1 ASC NULLS LAST], file_type=parquet, predicate=timeframe@0 = quarterly AND DynamicFilter [ empty ], pruning_predicate=timeframe_null_count@2 != row_count@3 AND timeframe_min@0 <= quarterly AND quarterly <= timeframe_max@1, required_guarantees=[timeframe in (quarterly)] @@ -487,8 +483,7 @@ LIMIT 3; ---- logical_plan 01)Sort: timeseries_parquet.period_end DESC NULLS FIRST, fetch=3 -02)--Filter: timeseries_parquet.timeframe = Utf8View("daily") OR timeseries_parquet.timeframe = Utf8View("weekly") -03)----TableScan: timeseries_parquet projection=[timeframe, period_end, value], partial_filters=[timeseries_parquet.timeframe = Utf8View("daily") OR timeseries_parquet.timeframe = Utf8View("weekly")] +02)--TableScan: timeseries_parquet projection=[timeframe, period_end, value], partial_filters=[timeseries_parquet.timeframe = Utf8View("daily") OR timeseries_parquet.timeframe = Utf8View("weekly")] physical_plan 01)SortExec: TopK(fetch=3), expr=[period_end@1 DESC], preserve_partitioning=[false] 02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/timeseries_sorted.parquet]]}, projection=[timeframe, period_end, value], output_ordering=[timeframe@0 ASC NULLS LAST, period_end@1 ASC NULLS LAST], file_type=parquet, predicate=(timeframe@0 = daily OR timeframe@0 = weekly) AND DynamicFilter [ empty ], pruning_predicate=timeframe_null_count@2 != row_count@3 AND timeframe_min@0 <= daily AND daily <= timeframe_max@1 OR timeframe_null_count@2 != row_count@3 AND timeframe_min@0 <= weekly AND weekly <= timeframe_max@1, required_guarantees=[timeframe in (daily, weekly)] @@ -507,8 +502,7 @@ LIMIT 2; ---- logical_plan 01)Sort: Utf8("constant") ASC NULLS LAST, timeseries_parquet.period_end DESC NULLS FIRST, fetch=2 -02)--Filter: timeseries_parquet.timeframe = Utf8View("monthly") -03)----TableScan: timeseries_parquet projection=[timeframe, period_end, value], partial_filters=[timeseries_parquet.timeframe = Utf8View("monthly")] +02)--TableScan: timeseries_parquet projection=[timeframe, period_end, value], partial_filters=[timeseries_parquet.timeframe = Utf8View("monthly")] physical_plan 01)SortExec: TopK(fetch=2), expr=[period_end@1 DESC], preserve_partitioning=[false] 02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/timeseries_sorted.parquet]]}, projection=[timeframe, period_end, value], file_type=parquet, predicate=timeframe@0 = monthly AND DynamicFilter [ empty ], reverse_row_groups=true, pruning_predicate=timeframe_null_count@2 != row_count@3 AND timeframe_min@0 <= monthly AND monthly <= timeframe_max@1, required_guarantees=[timeframe in (monthly)] @@ -536,8 +530,7 @@ ORDER BY timeframe DESC; ---- logical_plan 01)Sort: timeseries_parquet.timeframe DESC NULLS FIRST -02)--Filter: timeseries_parquet.period_end = Int32(2) -03)----TableScan: timeseries_parquet projection=[timeframe, period_end, value], partial_filters=[timeseries_parquet.period_end = Int32(2)] +02)--TableScan: timeseries_parquet projection=[timeframe, period_end, value], partial_filters=[timeseries_parquet.period_end = Int32(2)] physical_plan 01)SortExec: expr=[timeframe@0 DESC], preserve_partitioning=[false] 02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/timeseries_sorted.parquet]]}, projection=[timeframe, period_end, value], file_type=parquet, predicate=period_end@1 = 2, reverse_row_groups=true, pruning_predicate=period_end_null_count@2 != row_count@3 AND period_end_min@0 <= 2 AND 2 <= period_end_max@1, required_guarantees=[period_end in (2)] diff --git a/datafusion/sqllogictest/test_files/string/string_view.slt b/datafusion/sqllogictest/test_files/string/string_view.slt index 13b0aba653efb..2666a32941030 100644 --- a/datafusion/sqllogictest/test_files/string/string_view.slt +++ b/datafusion/sqllogictest/test_files/string/string_view.slt @@ -215,8 +215,7 @@ explain SELECT column1_utf8 from test where column1_utf8view = 'Andrew'; ---- logical_plan 01)Projection: test.column1_utf8 -02)--Filter: test.column1_utf8view = Utf8View("Andrew") -03)----TableScan: test projection=[column1_utf8, column1_utf8view] +02)--TableScan: test projection=[column1_utf8, column1_utf8view], unsupported_filters=[test.column1_utf8view = Utf8View("Andrew")] # reverse order should be the same query TT @@ -224,38 +223,31 @@ explain SELECT column1_utf8 from test where 'Andrew' = column1_utf8view; ---- logical_plan 01)Projection: test.column1_utf8 -02)--Filter: test.column1_utf8view = Utf8View("Andrew") -03)----TableScan: test projection=[column1_utf8, column1_utf8view] +02)--TableScan: test projection=[column1_utf8, column1_utf8view], unsupported_filters=[test.column1_utf8view = Utf8View("Andrew")] query TT explain SELECT column1_utf8 from test where column1_utf8 = arrow_cast('Andrew', 'Utf8View'); ---- -logical_plan -01)Filter: test.column1_utf8 = Utf8("Andrew") -02)--TableScan: test projection=[column1_utf8] +logical_plan TableScan: test projection=[column1_utf8], unsupported_filters=[test.column1_utf8 = Utf8("Andrew")] query TT explain SELECT column1_utf8 from test where arrow_cast('Andrew', 'Utf8View') = column1_utf8; ---- -logical_plan -01)Filter: test.column1_utf8 = Utf8("Andrew") -02)--TableScan: test projection=[column1_utf8] +logical_plan TableScan: test projection=[column1_utf8], unsupported_filters=[test.column1_utf8 = Utf8("Andrew")] query TT explain SELECT column1_utf8 from test where column1_utf8view = arrow_cast('Andrew', 'Dictionary(Int32, Utf8)'); ---- logical_plan 01)Projection: test.column1_utf8 -02)--Filter: test.column1_utf8view = Utf8View("Andrew") -03)----TableScan: test projection=[column1_utf8, column1_utf8view] +02)--TableScan: test projection=[column1_utf8, column1_utf8view], unsupported_filters=[test.column1_utf8view = Utf8View("Andrew")] query TT explain SELECT column1_utf8 from test where arrow_cast('Andrew', 'Dictionary(Int32, Utf8)') = column1_utf8view; ---- logical_plan 01)Projection: test.column1_utf8 -02)--Filter: test.column1_utf8view = Utf8View("Andrew") -03)----TableScan: test projection=[column1_utf8, column1_utf8view] +02)--TableScan: test projection=[column1_utf8, column1_utf8view], unsupported_filters=[test.column1_utf8view = Utf8View("Andrew")] # compare string / stringview # Should cast string -> stringview (which is cheap), not stringview -> string (which is not) @@ -264,16 +256,14 @@ explain SELECT column1_utf8 from test where column1_utf8view = column2_utf8; ---- logical_plan 01)Projection: test.column1_utf8 -02)--Filter: test.column1_utf8view = CAST(test.column2_utf8 AS Utf8View) -03)----TableScan: test projection=[column1_utf8, column2_utf8, column1_utf8view] +02)--TableScan: test projection=[column1_utf8, column2_utf8, column1_utf8view], unsupported_filters=[test.column1_utf8view = CAST(test.column2_utf8 AS Utf8View)] query TT explain SELECT column1_utf8 from test where column2_utf8 = column1_utf8view; ---- logical_plan 01)Projection: test.column1_utf8 -02)--Filter: CAST(test.column2_utf8 AS Utf8View) = test.column1_utf8view -03)----TableScan: test projection=[column1_utf8, column2_utf8, column1_utf8view] +02)--TableScan: test projection=[column1_utf8, column2_utf8, column1_utf8view], unsupported_filters=[CAST(test.column2_utf8 AS Utf8View) = test.column1_utf8view] query TT EXPLAIN SELECT diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index e73f4ec3e32da..4bd877b1749a8 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -388,12 +388,11 @@ query TT explain SELECT t1_id, t1_name, t1_int FROM t1 WHERE EXISTS(SELECT t1_int FROM t1 WHERE t1.t1_id > t1.t1_int) ---- logical_plan -01)LeftSemi Join: -02)--TableScan: t1 projection=[t1_id, t1_name, t1_int] +01)LeftSemi Join: +02)--TableScan: t1 projection=[t1_id, t1_name, t1_int], unsupported_filters=[Boolean(true)] 03)--SubqueryAlias: __correlated_sq_1 04)----Projection: -05)------Filter: t1.t1_int < t1.t1_id -06)--------TableScan: t1 projection=[t1_id, t1_int] +05)------TableScan: t1 projection=[t1_id, t1_int], unsupported_filters=[t1.t1_int < t1.t1_id] #in_subquery_with_same_table @@ -407,8 +406,7 @@ logical_plan 02)--TableScan: t1 projection=[t1_id, t1_name, t1_int] 03)--SubqueryAlias: __correlated_sq_1 04)----Projection: t1.t1_int -05)------Filter: t1.t1_int < t1.t1_id -06)--------TableScan: t1 projection=[t1_id, t1_int] +05)------TableScan: t1 projection=[t1_int, t1_id], unsupported_filters=[t1.t1_int < t1.t1_id] #in_subquery_nested_exist_subquery query TT @@ -464,9 +462,8 @@ logical_plan 02)--Subquery: 03)----Projection: t2.t2_int 04)------Limit: skip=0, fetch=1 -05)--------Filter: t2.t2_int = outer_ref(t1.t1_int) -06)----------TableScan: t2 -07)--TableScan: t1 projection=[t1_id, t1_int] +05)--------TableScan: t2, unsupported_filters=[t2.t2_int = outer_ref(t1.t1_int)], fetch=1 +06)--TableScan: t1 projection=[t1_id, t1_int] query TT explain SELECT t1_id from t1 where t1_int = (SELECT t2_int FROM t2 WHERE t2.t2_int = t1.t1_int limit 1) @@ -477,9 +474,8 @@ logical_plan 03)----Subquery: 04)------Projection: t2.t2_int 05)--------Limit: skip=0, fetch=1 -06)----------Filter: t2.t2_int = outer_ref(t1.t1_int) -07)------------TableScan: t2 -08)----TableScan: t1 projection=[t1_id, t1_int] +06)----------TableScan: t2, unsupported_filters=[t2.t2_int = outer_ref(t1.t1_int)], fetch=1 +07)----TableScan: t1 projection=[t1_id, t1_int] query TT explain SELECT t1_id, (SELECT a FROM (select 1 as a) WHERE a = t1.t1_int) as t2_int from t1 @@ -510,9 +506,8 @@ logical_plan 02)--Subquery: 03)----Projection: sum(t2.t2_int) 04)------Aggregate: groupBy=[[]], aggr=[[sum(CAST(t2.t2_int AS Int64))]] -05)--------Filter: t2.t2_id < outer_ref(t1.t1_id) -06)----------TableScan: t2 -07)--TableScan: t1 projection=[t1_id] +05)--------TableScan: t2, unsupported_filters=[t2.t2_id < outer_ref(t1.t1_id)] +06)--TableScan: t1 projection=[t1_id] #aggregated_correlated_scalar_subquery_with_extra_group_by_columns statement error DataFusion error: Invalid \(non-executable\) plan after Analyzer\ncaused by\nError during planning: A GROUP BY clause in a scalar correlated subquery cannot contain non-correlated columns @@ -524,13 +519,11 @@ explain SELECT t1_id, t1_name FROM t1 WHERE EXISTS (SELECT sum(t1.t1_int + t2.t2 ---- logical_plan 01)Projection: t1.t1_id, t1.t1_name -02)--Filter: EXISTS () +02)--TableScan: t1 projection=[t1_id, t1_name, t1_int], unsupported_filters=[EXISTS ()] 03)----Subquery: 04)------Projection: sum(outer_ref(t1.t1_int) + t2.t2_id) 05)--------Aggregate: groupBy=[[]], aggr=[[sum(CAST(outer_ref(t1.t1_int) + t2.t2_id AS Int64))]] -06)----------Filter: outer_ref(t1.t1_name) = t2.t2_name -07)------------TableScan: t2 -08)----TableScan: t1 projection=[t1_id, t1_name, t1_int] +06)----------TableScan: t2, unsupported_filters=[outer_ref(t1.t1_name) = t2.t2_name] #support_agg_correlated_columns2 query TT @@ -538,27 +531,24 @@ explain SELECT t1_id, t1_name FROM t1 WHERE EXISTS (SELECT count(*) FROM t2 WHER ---- logical_plan 01)Projection: t1.t1_id, t1.t1_name -02)--Filter: EXISTS () +02)--TableScan: t1 projection=[t1_id, t1_name, t1_int], unsupported_filters=[EXISTS ()] 03)----Subquery: 04)------Projection: count(Int64(1)) AS count(*) 05)--------Filter: sum(outer_ref(t1.t1_int) + t2.t2_id) > Int64(0) 06)----------Aggregate: groupBy=[[]], aggr=[[count(Int64(1)), sum(CAST(outer_ref(t1.t1_int) + t2.t2_id AS Int64))]] -07)------------Filter: outer_ref(t1.t1_name) = t2.t2_name -08)--------------TableScan: t2 -09)----TableScan: t1 projection=[t1_id, t1_name, t1_int] +07)------------TableScan: t2, unsupported_filters=[outer_ref(t1.t1_name) = t2.t2_name] #support_join_correlated_columns query TT explain SELECT t0_id, t0_name FROM t0 WHERE EXISTS (SELECT 1 FROM t1 INNER JOIN t2 ON(t1.t1_id = t2.t2_id and t1.t1_name = t0.t0_name)) ---- logical_plan -01)LeftSemi Join: t0.t0_name = __correlated_sq_2.t1_name -02)--TableScan: t0 projection=[t0_id, t0_name] -03)--SubqueryAlias: __correlated_sq_2 -04)----Projection: t1.t1_name -05)------Inner Join: t1.t1_id = t2.t2_id -06)--------TableScan: t1 projection=[t1_id, t1_name] -07)--------TableScan: t2 projection=[t2_id] +01)TableScan: t0 projection=[t0_id, t0_name], unsupported_filters=[EXISTS ()] +02)--Subquery: +03)----Projection: Int64(1) +04)------Inner Join: t1.t1_id = t2.t2_id +05)--------TableScan: t1, unsupported_filters=[t1.t1_name = outer_ref(t0.t0_name)] +06)--------TableScan: t2 #subquery_contains_join_contains_correlated_columns query TT @@ -594,21 +584,19 @@ query TT explain SELECT t1_id, t1_name FROM t1 WHERE EXISTS (SELECT * FROM t2 WHERE t2_id >= t1_id order by t1_id) ---- logical_plan -01)Filter: EXISTS () +01)TableScan: t1 projection=[t1_id, t1_name], unsupported_filters=[EXISTS ()] 02)--Subquery: 03)----Sort: outer_ref(t1.t1_id) ASC NULLS LAST 04)------Projection: t2.t2_id, t2.t2_name, t2.t2_int -05)--------Filter: t2.t2_id >= outer_ref(t1.t1_id) -06)----------TableScan: t2 -07)--TableScan: t1 projection=[t1_id, t1_name] +05)--------TableScan: t2, unsupported_filters=[t2.t2_id >= outer_ref(t1.t1_id)] #exists_subquery_with_select_null query TT explain SELECT t1_id, t1_name FROM t1 WHERE EXISTS (SELECT NULL) ---- logical_plan -01)LeftSemi Join: -02)--TableScan: t1 projection=[t1_id, t1_name] +01)LeftSemi Join: +02)--TableScan: t1 projection=[t1_id, t1_name], unsupported_filters=[Boolean(true)] 03)--SubqueryAlias: __correlated_sq_1 04)----EmptyRelation: rows=1 @@ -663,13 +651,11 @@ query TT explain SELECT t1_id, t1_name FROM t1 WHERE t1_id in (SELECT t2_id FROM t2 where t1_name = t2_name limit 10) ---- logical_plan -01)Filter: t1.t1_id IN () +01)TableScan: t1 projection=[t1_id, t1_name], unsupported_filters=[t1.t1_id IN ()] 02)--Subquery: 03)----Projection: t2.t2_id 04)------Limit: skip=0, fetch=10 -05)--------Filter: outer_ref(t1.t1_name) = t2.t2_name -06)----------TableScan: t2 -07)--TableScan: t1 projection=[t1_id, t1_name] +05)--------TableScan: t2, unsupported_filters=[outer_ref(t1.t1_name) = t2.t2_name], fetch=10 #in_non_correlated_subquery_with_limit #de-correlated, limit is kept @@ -707,16 +693,13 @@ query TT explain SELECT t1_id, t1_name FROM t1 WHERE EXISTS (SELECT * FROM t2 WHERE t2_id = t1_id UNION ALL SELECT * FROM t2 WHERE upper(t2_name) = upper(t1.t1_name)) ---- logical_plan -01)Filter: EXISTS () +01)TableScan: t1 projection=[t1_id, t1_name], unsupported_filters=[EXISTS ()] 02)--Subquery: 03)----Union 04)------Projection: t2.t2_id, t2.t2_name, t2.t2_int -05)--------Filter: t2.t2_id = outer_ref(t1.t1_id) -06)----------TableScan: t2 -07)------Projection: t2.t2_id, t2.t2_name, t2.t2_int -08)--------Filter: upper(t2.t2_name) = upper(outer_ref(t1.t1_name)) -09)----------TableScan: t2 -10)--TableScan: t1 projection=[t1_id, t1_name] +05)--------TableScan: t2, unsupported_filters=[t2.t2_id = outer_ref(t1.t1_id)] +06)------Projection: t2.t2_id, t2.t2_name, t2.t2_int +07)--------TableScan: t2, unsupported_filters=[upper(t2.t2_name) = upper(outer_ref(t1.t1_name))] #simple_uncorrelated_scalar_subquery query TT @@ -735,14 +718,14 @@ explain select (select count(*) from t1) as b, (select count(1) from t2) ---- logical_plan 01)Projection: __scalar_sq_1.count(*) AS b, __scalar_sq_2.count(Int64(1)) AS count(Int64(1)) -02)--Left Join: +02)--Left Join: 03)----SubqueryAlias: __scalar_sq_1 04)------Projection: count(Int64(1)) AS count(*) 05)--------Aggregate: groupBy=[[]], aggr=[[count(Int64(1))]] 06)----------TableScan: t1 projection=[] 07)----SubqueryAlias: __scalar_sq_2 08)------Aggregate: groupBy=[[]], aggr=[[count(Int64(1))]] -09)--------TableScan: t2 projection=[] +09)--------TableScan: t2 projection=[], unsupported_filters=[Boolean(true)] statement ok set datafusion.explain.logical_plan_only = false; @@ -752,21 +735,24 @@ explain select (select count(*) from t1) as b, (select count(1) from t2) ---- logical_plan 01)Projection: __scalar_sq_1.count(*) AS b, __scalar_sq_2.count(Int64(1)) AS count(Int64(1)) -02)--Left Join: +02)--Left Join: 03)----SubqueryAlias: __scalar_sq_1 04)------Projection: count(Int64(1)) AS count(*) 05)--------Aggregate: groupBy=[[]], aggr=[[count(Int64(1))]] 06)----------TableScan: t1 projection=[] 07)----SubqueryAlias: __scalar_sq_2 08)------Aggregate: groupBy=[[]], aggr=[[count(Int64(1))]] -09)--------TableScan: t2 projection=[] +09)--------TableScan: t2 projection=[], unsupported_filters=[Boolean(true)] physical_plan 01)ProjectionExec: expr=[count(*)@0 as b, count(Int64(1))@1 as count(Int64(1))] 02)--NestedLoopJoinExec: join_type=Left 03)----ProjectionExec: expr=[4 as count(*)] 04)------PlaceholderRowExec -05)----ProjectionExec: expr=[4 as count(Int64(1))] -06)------PlaceholderRowExec +05)----AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] +06)------CoalescePartitionsExec +07)--------AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] +08)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)------------DataSourceExec: partitions=1, partition_sizes=[2] statement ok set datafusion.explain.logical_plan_only = true; @@ -1439,8 +1425,8 @@ query TT explain SELECT a FROM t1 WHERE EXISTS (SELECT count(*) FROM t2) ---- logical_plan -01)LeftSemi Join: -02)--TableScan: t1 projection=[a] +01)LeftSemi Join: +02)--TableScan: t1 projection=[a], unsupported_filters=[Boolean(true)] 03)--SubqueryAlias: __correlated_sq_1 04)----EmptyRelation: rows=1 diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part index 84237e4393377..3d872d3b76aa9 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part @@ -44,8 +44,7 @@ logical_plan 02)--Projection: lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity) AS sum_qty, sum(lineitem.l_extendedprice) AS sum_base_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS sum_disc_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax) AS sum_charge, avg(lineitem.l_quantity) AS avg_qty, avg(lineitem.l_extendedprice) AS avg_price, avg(lineitem.l_discount) AS avg_disc, count(Int64(1)) AS count(*) AS count_order 03)----Aggregate: groupBy=[[lineitem.l_returnflag, lineitem.l_linestatus]], aggr=[[sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum(__common_expr_1) AS sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(__common_expr_1 * (Decimal128(Some(1),20,0) + lineitem.l_tax)) AS sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), avg(lineitem.l_quantity), avg(lineitem.l_extendedprice), avg(lineitem.l_discount), count(Int64(1))]] 04)------Projection: lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount) AS __common_expr_1, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus -05)--------Filter: lineitem.l_shipdate <= Date32("1998-09-02") -06)----------TableScan: lineitem projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], partial_filters=[lineitem.l_shipdate <= Date32("1998-09-02")] +05)--------TableScan: lineitem projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], partial_filters=[lineitem.l_shipdate <= Date32("1998-09-02")] physical_plan 01)SortPreservingMergeExec: [l_returnflag@0 ASC NULLS LAST, l_linestatus@1 ASC NULLS LAST] 02)--SortExec: expr=[l_returnflag@0 ASC NULLS LAST, l_linestatus@1 ASC NULLS LAST], preserve_partitioning=[true] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part index 62649148bf058..05ffc18353cc9 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part @@ -62,12 +62,10 @@ logical_plan 09)----------------Inner Join: customer.c_custkey = orders.o_custkey 10)------------------TableScan: customer projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment] 11)------------------Projection: orders.o_orderkey, orders.o_custkey -12)--------------------Filter: orders.o_orderdate >= Date32("1993-10-01") AND orders.o_orderdate < Date32("1994-01-01") -13)----------------------TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate], partial_filters=[orders.o_orderdate >= Date32("1993-10-01"), orders.o_orderdate < Date32("1994-01-01")] -14)--------------Projection: lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount -15)----------------Filter: lineitem.l_returnflag = Utf8View("R") -16)------------------TableScan: lineitem projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag], partial_filters=[lineitem.l_returnflag = Utf8View("R")] -17)----------TableScan: nation projection=[n_nationkey, n_name] +12)--------------------TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate], partial_filters=[orders.o_orderdate >= Date32("1993-10-01"), orders.o_orderdate < Date32("1994-01-01")] +13)--------------Projection: lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount +14)----------------TableScan: lineitem projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag], partial_filters=[lineitem.l_returnflag = Utf8View("R")] +15)----------TableScan: nation projection=[n_nationkey, n_name] physical_plan 01)SortPreservingMergeExec: [revenue@2 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[revenue@2 DESC], preserve_partitioning=[true] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part index a31579eb1e09d..58d8eec35ce52 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part @@ -58,20 +58,18 @@ logical_plan 09)----------------TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost], partial_filters=[Boolean(true)] 10)----------------TableScan: supplier projection=[s_suppkey, s_nationkey] 11)------------Projection: nation.n_nationkey -12)--------------Filter: nation.n_name = Utf8View("GERMANY") -13)----------------TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8View("GERMANY")] -14)------SubqueryAlias: __scalar_sq_1 -15)--------Projection: CAST(CAST(sum(partsupp.ps_supplycost * partsupp.ps_availqty) AS Float64) * Float64(0.0001) AS Decimal128(38, 15)) -16)----------Aggregate: groupBy=[[]], aggr=[[sum(partsupp.ps_supplycost * CAST(partsupp.ps_availqty AS Decimal128(10, 0)))]] -17)------------Projection: partsupp.ps_availqty, partsupp.ps_supplycost -18)--------------Inner Join: supplier.s_nationkey = nation.n_nationkey -19)----------------Projection: partsupp.ps_availqty, partsupp.ps_supplycost, supplier.s_nationkey -20)------------------Inner Join: partsupp.ps_suppkey = supplier.s_suppkey -21)--------------------TableScan: partsupp projection=[ps_suppkey, ps_availqty, ps_supplycost] -22)--------------------TableScan: supplier projection=[s_suppkey, s_nationkey] -23)----------------Projection: nation.n_nationkey -24)------------------Filter: nation.n_name = Utf8View("GERMANY") -25)--------------------TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8View("GERMANY")] +12)--------------TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8View("GERMANY")] +13)------SubqueryAlias: __scalar_sq_1 +14)--------Projection: CAST(CAST(sum(partsupp.ps_supplycost * partsupp.ps_availqty) AS Float64) * Float64(0.0001) AS Decimal128(38, 15)) +15)----------Aggregate: groupBy=[[]], aggr=[[sum(partsupp.ps_supplycost * CAST(partsupp.ps_availqty AS Decimal128(10, 0)))]] +16)------------Projection: partsupp.ps_availqty, partsupp.ps_supplycost +17)--------------Inner Join: supplier.s_nationkey = nation.n_nationkey +18)----------------Projection: partsupp.ps_availqty, partsupp.ps_supplycost, supplier.s_nationkey +19)------------------Inner Join: partsupp.ps_suppkey = supplier.s_suppkey +20)--------------------TableScan: partsupp projection=[ps_suppkey, ps_availqty, ps_supplycost] +21)--------------------TableScan: supplier projection=[s_suppkey, s_nationkey] +22)----------------Projection: nation.n_nationkey +23)------------------TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8View("GERMANY")] physical_plan 01)SortExec: TopK(fetch=10), expr=[value@1 DESC], preserve_partitioning=[false] 02)--ProjectionExec: expr=[ps_partkey@0 as ps_partkey, sum(partsupp.ps_supplycost * partsupp.ps_availqty)@1 as value] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part index b152fde02f060..cee730cae0a4c 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part @@ -55,9 +55,8 @@ logical_plan 04)------Projection: lineitem.l_shipmode, orders.o_orderpriority 05)--------Inner Join: lineitem.l_orderkey = orders.o_orderkey 06)----------Projection: lineitem.l_orderkey, lineitem.l_shipmode -07)------------Filter: (lineitem.l_shipmode = Utf8View("MAIL") OR lineitem.l_shipmode = Utf8View("SHIP")) AND lineitem.l_receiptdate > lineitem.l_commitdate AND lineitem.l_shipdate < lineitem.l_commitdate AND lineitem.l_receiptdate >= Date32("1994-01-01") AND lineitem.l_receiptdate < Date32("1995-01-01") -08)--------------TableScan: lineitem projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], partial_filters=[lineitem.l_shipmode = Utf8View("MAIL") OR lineitem.l_shipmode = Utf8View("SHIP"), lineitem.l_receiptdate > lineitem.l_commitdate, lineitem.l_shipdate < lineitem.l_commitdate, lineitem.l_receiptdate >= Date32("1994-01-01"), lineitem.l_receiptdate < Date32("1995-01-01")] -09)----------TableScan: orders projection=[o_orderkey, o_orderpriority] +07)------------TableScan: lineitem projection=[l_orderkey, l_shipmode, l_shipdate, l_commitdate, l_receiptdate], partial_filters=[lineitem.l_shipmode = Utf8View("MAIL") OR lineitem.l_shipmode = Utf8View("SHIP"), lineitem.l_receiptdate > lineitem.l_commitdate, lineitem.l_shipdate < lineitem.l_commitdate, lineitem.l_receiptdate >= Date32("1994-01-01"), lineitem.l_receiptdate < Date32("1995-01-01")] +08)----------TableScan: orders projection=[o_orderkey, o_orderpriority] physical_plan 01)SortPreservingMergeExec: [l_shipmode@0 ASC NULLS LAST] 02)--SortExec: expr=[l_shipmode@0 ASC NULLS LAST], preserve_partitioning=[true] @@ -67,7 +66,7 @@ physical_plan 06)----------AggregateExec: mode=Partial, gby=[l_shipmode@0 as l_shipmode], aggr=[sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] 07)------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_shipmode@1, o_orderpriority@3] 08)--------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -09)----------------FilterExec: (l_shipmode@4 = MAIL OR l_shipmode@4 = SHIP) AND l_receiptdate@3 > l_commitdate@2 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 1994-01-01 AND l_receiptdate@3 < 1995-01-01, projection=[l_orderkey@0, l_shipmode@4] -10)------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], file_type=csv, has_header=false +09)----------------FilterExec: (l_shipmode@1 = MAIL OR l_shipmode@1 = SHIP) AND l_receiptdate@4 > l_commitdate@3 AND l_shipdate@2 < l_commitdate@3 AND l_receiptdate@4 >= 1994-01-01 AND l_receiptdate@4 < 1995-01-01, projection=[l_orderkey@0, l_shipmode@1] +10)------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_shipmode, l_shipdate, l_commitdate, l_receiptdate], file_type=csv, has_header=false 11)--------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 12)----------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderpriority], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part index 94e0848bfcce1..3c804d9b1f600 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part @@ -50,8 +50,7 @@ logical_plan 08)--------------Left Join: customer.c_custkey = orders.o_custkey 09)----------------TableScan: customer projection=[c_custkey] 10)----------------Projection: orders.o_orderkey, orders.o_custkey -11)------------------Filter: orders.o_comment NOT LIKE Utf8View("%special%requests%") -12)--------------------TableScan: orders projection=[o_orderkey, o_custkey, o_comment], partial_filters=[orders.o_comment NOT LIKE Utf8View("%special%requests%")] +11)------------------TableScan: orders projection=[o_orderkey, o_custkey, o_comment], partial_filters=[orders.o_comment NOT LIKE Utf8View("%special%requests%")] physical_plan 01)SortPreservingMergeExec: [custdist@1 DESC, c_count@0 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[custdist@1 DESC, c_count@0 DESC], preserve_partitioning=[true] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part index a9ac517f287d0..cf5ac5a9d8bac 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part @@ -37,9 +37,8 @@ logical_plan 03)----Projection: lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount) AS __common_expr_1, part.p_type 04)------Inner Join: lineitem.l_partkey = part.p_partkey 05)--------Projection: lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount -06)----------Filter: lineitem.l_shipdate >= Date32("1995-09-01") AND lineitem.l_shipdate < Date32("1995-10-01") -07)------------TableScan: lineitem projection=[l_partkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("1995-09-01"), lineitem.l_shipdate < Date32("1995-10-01")] -08)--------TableScan: part projection=[p_partkey, p_type] +06)----------TableScan: lineitem projection=[l_partkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("1995-09-01"), lineitem.l_shipdate < Date32("1995-10-01")] +07)--------TableScan: part projection=[p_partkey, p_type] physical_plan 01)ProjectionExec: expr=[100 * CAST(sum(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END)@0 AS Float64) / CAST(sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 AS Float64) as promo_revenue] 02)--AggregateExec: mode=Final, gby=[], aggr=[sum(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part index ae0c0a93a3552..5352ae17a73f5 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part @@ -60,16 +60,14 @@ logical_plan 08)------------Projection: lineitem.l_suppkey AS supplier_no, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS total_revenue 09)--------------Aggregate: groupBy=[[lineitem.l_suppkey]], aggr=[[sum(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)) AS sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] 10)----------------Projection: lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount -11)------------------Filter: lineitem.l_shipdate >= Date32("1996-01-01") AND lineitem.l_shipdate < Date32("1996-04-01") -12)--------------------TableScan: lineitem projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("1996-01-01"), lineitem.l_shipdate < Date32("1996-04-01")] -13)------SubqueryAlias: __scalar_sq_1 -14)--------Aggregate: groupBy=[[]], aggr=[[max(revenue0.total_revenue)]] -15)----------SubqueryAlias: revenue0 -16)------------Projection: sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS total_revenue -17)--------------Aggregate: groupBy=[[lineitem.l_suppkey]], aggr=[[sum(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)) AS sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] -18)----------------Projection: lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount -19)------------------Filter: lineitem.l_shipdate >= Date32("1996-01-01") AND lineitem.l_shipdate < Date32("1996-04-01") -20)--------------------TableScan: lineitem projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("1996-01-01"), lineitem.l_shipdate < Date32("1996-04-01")] +11)------------------TableScan: lineitem projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("1996-01-01"), lineitem.l_shipdate < Date32("1996-04-01")] +12)------SubqueryAlias: __scalar_sq_1 +13)--------Aggregate: groupBy=[[]], aggr=[[max(revenue0.total_revenue)]] +14)----------SubqueryAlias: revenue0 +15)------------Projection: sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS total_revenue +16)--------------Aggregate: groupBy=[[lineitem.l_suppkey]], aggr=[[sum(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)) AS sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] +17)----------------Projection: lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount +18)------------------TableScan: lineitem projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("1996-01-01"), lineitem.l_shipdate < Date32("1996-04-01")] physical_plan 01)SortPreservingMergeExec: [s_suppkey@0 ASC NULLS LAST] 02)--SortExec: expr=[s_suppkey@0 ASC NULLS LAST], preserve_partitioning=[true] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part index b01110b567ca8..b13f153123de1 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part @@ -58,12 +58,10 @@ logical_plan 06)----------Projection: partsupp.ps_suppkey, part.p_brand, part.p_type, part.p_size 07)------------Inner Join: partsupp.ps_partkey = part.p_partkey 08)--------------TableScan: partsupp projection=[ps_partkey, ps_suppkey] -09)--------------Filter: part.p_brand != Utf8View("Brand#45") AND part.p_type NOT LIKE Utf8View("MEDIUM POLISHED%") AND part.p_size IN ([Int32(49), Int32(14), Int32(23), Int32(45), Int32(19), Int32(3), Int32(36), Int32(9)]) -10)----------------TableScan: part projection=[p_partkey, p_brand, p_type, p_size], partial_filters=[part.p_brand != Utf8View("Brand#45"), part.p_type NOT LIKE Utf8View("MEDIUM POLISHED%"), part.p_size IN ([Int32(49), Int32(14), Int32(23), Int32(45), Int32(19), Int32(3), Int32(36), Int32(9)])] -11)----------SubqueryAlias: __correlated_sq_1 -12)------------Projection: supplier.s_suppkey -13)--------------Filter: supplier.s_comment LIKE Utf8View("%Customer%Complaints%") -14)----------------TableScan: supplier projection=[s_suppkey, s_comment], partial_filters=[supplier.s_comment LIKE Utf8View("%Customer%Complaints%")] +09)--------------TableScan: part projection=[p_partkey, p_brand, p_type, p_size], partial_filters=[part.p_brand != Utf8View("Brand#45"), part.p_type NOT LIKE Utf8View("MEDIUM POLISHED%"), part.p_size IN ([Int32(49), Int32(14), Int32(23), Int32(45), Int32(19), Int32(3), Int32(36), Int32(9)])] +10)----------SubqueryAlias: __correlated_sq_1 +11)------------Projection: supplier.s_suppkey +12)--------------TableScan: supplier projection=[s_suppkey, s_comment], partial_filters=[supplier.s_comment LIKE Utf8View("%Customer%Complaints%")] physical_plan 01)SortPreservingMergeExec: [supplier_cnt@3 DESC, p_brand@0 ASC NULLS LAST, p_type@1 ASC NULLS LAST, p_size@2 ASC NULLS LAST], fetch=10 02)--SortExec: TopK(fetch=10), expr=[supplier_cnt@3 DESC, p_brand@0 ASC NULLS LAST, p_type@1 ASC NULLS LAST, p_size@2 ASC NULLS LAST], preserve_partitioning=[true] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part index 83294d61a1698..08257099b1a63 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part @@ -44,12 +44,11 @@ logical_plan 06)----------Inner Join: lineitem.l_partkey = part.p_partkey 07)------------TableScan: lineitem projection=[l_partkey, l_quantity, l_extendedprice] 08)------------Projection: part.p_partkey -09)--------------Filter: part.p_brand = Utf8View("Brand#23") AND part.p_container = Utf8View("MED BOX") -10)----------------TableScan: part projection=[p_partkey, p_brand, p_container], partial_filters=[part.p_brand = Utf8View("Brand#23"), part.p_container = Utf8View("MED BOX")] -11)--------SubqueryAlias: __scalar_sq_1 -12)----------Projection: CAST(Float64(0.2) * CAST(avg(lineitem.l_quantity) AS Float64) AS Decimal128(30, 15)), lineitem.l_partkey -13)------------Aggregate: groupBy=[[lineitem.l_partkey]], aggr=[[avg(lineitem.l_quantity)]] -14)--------------TableScan: lineitem projection=[l_partkey, l_quantity] +09)--------------TableScan: part projection=[p_partkey, p_brand, p_container], partial_filters=[part.p_brand = Utf8View("Brand#23"), part.p_container = Utf8View("MED BOX")] +10)--------SubqueryAlias: __scalar_sq_1 +11)----------Projection: CAST(Float64(0.2) * CAST(avg(lineitem.l_quantity) AS Float64) AS Decimal128(30, 15)), lineitem.l_partkey +12)------------Aggregate: groupBy=[[lineitem.l_partkey]], aggr=[[avg(lineitem.l_quantity)]] +13)--------------TableScan: lineitem projection=[l_partkey, l_quantity] physical_plan 01)ProjectionExec: expr=[CAST(sum(lineitem.l_extendedprice)@0 AS Float64) / 7 as avg_yearly] 02)--AggregateExec: mode=Final, gby=[], aggr=[sum(lineitem.l_extendedprice)] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part index 72c21e060fa66..cf5e078d4621d 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part @@ -59,10 +59,8 @@ logical_plan 03)----Projection: lineitem.l_extendedprice, lineitem.l_discount 04)------Inner Join: lineitem.l_partkey = part.p_partkey Filter: part.p_brand = Utf8View("Brand#12") AND part.p_container IN ([Utf8View("SM CASE"), Utf8View("SM BOX"), Utf8View("SM PACK"), Utf8View("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 = Utf8View("Brand#23") AND part.p_container IN ([Utf8View("MED BAG"), Utf8View("MED BOX"), Utf8View("MED PKG"), Utf8View("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 = Utf8View("Brand#34") AND part.p_container IN ([Utf8View("LG CASE"), Utf8View("LG BOX"), Utf8View("LG PACK"), Utf8View("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) 05)--------Projection: lineitem.l_partkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount -06)----------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 = Utf8View("AIR") OR lineitem.l_shipmode = Utf8View("AIR REG")) AND lineitem.l_shipinstruct = Utf8View("DELIVER IN PERSON") -07)------------TableScan: lineitem projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode], partial_filters=[lineitem.l_shipmode = Utf8View("AIR") OR lineitem.l_shipmode = Utf8View("AIR REG"), lineitem.l_shipinstruct = Utf8View("DELIVER IN PERSON"), 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)] -08)--------Filter: (part.p_brand = Utf8View("Brand#12") AND part.p_container IN ([Utf8View("SM CASE"), Utf8View("SM BOX"), Utf8View("SM PACK"), Utf8View("SM PKG")]) AND part.p_size <= Int32(5) OR part.p_brand = Utf8View("Brand#23") AND part.p_container IN ([Utf8View("MED BAG"), Utf8View("MED BOX"), Utf8View("MED PKG"), Utf8View("MED PACK")]) AND part.p_size <= Int32(10) OR part.p_brand = Utf8View("Brand#34") AND part.p_container IN ([Utf8View("LG CASE"), Utf8View("LG BOX"), Utf8View("LG PACK"), Utf8View("LG PKG")]) AND part.p_size <= Int32(15)) AND part.p_size >= Int32(1) -09)----------TableScan: part projection=[p_partkey, p_brand, p_size, p_container], partial_filters=[part.p_size >= Int32(1), part.p_brand = Utf8View("Brand#12") AND part.p_container IN ([Utf8View("SM CASE"), Utf8View("SM BOX"), Utf8View("SM PACK"), Utf8View("SM PKG")]) AND part.p_size <= Int32(5) OR part.p_brand = Utf8View("Brand#23") AND part.p_container IN ([Utf8View("MED BAG"), Utf8View("MED BOX"), Utf8View("MED PKG"), Utf8View("MED PACK")]) AND part.p_size <= Int32(10) OR part.p_brand = Utf8View("Brand#34") AND part.p_container IN ([Utf8View("LG CASE"), Utf8View("LG BOX"), Utf8View("LG PACK"), Utf8View("LG PKG")]) AND part.p_size <= Int32(15)] +06)----------TableScan: lineitem projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode], partial_filters=[lineitem.l_shipmode = Utf8View("AIR") OR lineitem.l_shipmode = Utf8View("AIR REG"), lineitem.l_shipinstruct = Utf8View("DELIVER IN PERSON"), 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)] +07)--------TableScan: part projection=[p_partkey, p_brand, p_size, p_container], partial_filters=[part.p_size >= Int32(1), part.p_brand = Utf8View("Brand#12") AND part.p_container IN ([Utf8View("SM CASE"), Utf8View("SM BOX"), Utf8View("SM PACK"), Utf8View("SM PKG")]) AND part.p_size <= Int32(5) OR part.p_brand = Utf8View("Brand#23") AND part.p_container IN ([Utf8View("MED BAG"), Utf8View("MED BOX"), Utf8View("MED PKG"), Utf8View("MED PACK")]) AND part.p_size <= Int32(10) OR part.p_brand = Utf8View("Brand#34") AND part.p_container IN ([Utf8View("LG CASE"), Utf8View("LG BOX"), Utf8View("LG PACK"), Utf8View("LG PKG")]) AND part.p_size <= Int32(15)] physical_plan 01)ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@0 as revenue] 02)--AggregateExec: mode=Final, gby=[], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] @@ -70,9 +68,9 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 05)--------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], filter=p_brand@1 = Brand#12 AND p_container@3 IN (SET) ([SM CASE, SM BOX, SM PACK, SM PKG]) AND l_quantity@0 >= Some(100),15,2 AND l_quantity@0 <= Some(1100),15,2 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_container@3 IN (SET) ([MED BAG, MED BOX, MED PKG, MED PACK]) AND l_quantity@0 >= Some(1000),15,2 AND l_quantity@0 <= Some(2000),15,2 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_container@3 IN (SET) ([LG CASE, LG BOX, LG PACK, LG PKG]) AND l_quantity@0 >= Some(2000),15,2 AND l_quantity@0 <= Some(3000),15,2 AND p_size@2 <= 15, projection=[l_extendedprice@2, l_discount@3] 06)----------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -07)------------FilterExec: (l_quantity@1 >= Some(100),15,2 AND l_quantity@1 <= Some(1100),15,2 OR l_quantity@1 >= Some(1000),15,2 AND l_quantity@1 <= Some(2000),15,2 OR l_quantity@1 >= Some(2000),15,2 AND l_quantity@1 <= Some(3000),15,2) AND (l_shipmode@5 = AIR OR l_shipmode@5 = AIR REG) AND l_shipinstruct@4 = DELIVER IN PERSON, projection=[l_partkey@0, l_quantity@1, l_extendedprice@2, l_discount@3] +07)------------FilterExec: (l_shipmode@5 = AIR OR l_shipmode@5 = AIR REG) AND l_shipinstruct@4 = DELIVER IN PERSON AND (l_quantity@1 >= Some(100),15,2 AND l_quantity@1 <= Some(1100),15,2 OR l_quantity@1 >= Some(1000),15,2 AND l_quantity@1 <= Some(2000),15,2 OR l_quantity@1 >= Some(2000),15,2 AND l_quantity@1 <= Some(3000),15,2), projection=[l_partkey@0, l_quantity@1, l_extendedprice@2, l_discount@3] 08)--------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode], file_type=csv, has_header=false 09)----------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -10)------------FilterExec: (p_brand@1 = Brand#12 AND p_container@3 IN (SET) ([SM CASE, SM BOX, SM PACK, SM PKG]) AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_container@3 IN (SET) ([MED BAG, MED BOX, MED PKG, MED PACK]) AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_container@3 IN (SET) ([LG CASE, LG BOX, LG PACK, LG PKG]) AND p_size@2 <= 15) AND p_size@2 >= 1 +10)------------FilterExec: p_size@2 >= 1 AND (p_brand@1 = Brand#12 AND p_container@3 IN (SET) ([SM CASE, SM BOX, SM PACK, SM PKG]) AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_container@3 IN (SET) ([MED BAG, MED BOX, MED PKG, MED PACK]) AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_container@3 IN (SET) ([LG CASE, LG BOX, LG PACK, LG PKG]) AND p_size@2 <= 15) 11)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 12)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_size, p_container], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part index d5ff6724402ad..c92239d050bad 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part @@ -75,29 +75,26 @@ logical_plan 10)------------------Projection: part.p_partkey, part.p_mfgr, partsupp.ps_suppkey, partsupp.ps_supplycost 11)--------------------Inner Join: part.p_partkey = partsupp.ps_partkey 12)----------------------Projection: part.p_partkey, part.p_mfgr -13)------------------------Filter: part.p_size = Int32(15) AND part.p_type LIKE Utf8View("%BRASS") -14)--------------------------TableScan: part projection=[p_partkey, p_mfgr, p_type, p_size], partial_filters=[part.p_size = Int32(15), part.p_type LIKE Utf8View("%BRASS")] -15)----------------------TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_supplycost] -16)------------------TableScan: supplier projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment] -17)--------------TableScan: nation projection=[n_nationkey, n_name, n_regionkey] -18)----------Projection: region.r_regionkey -19)------------Filter: region.r_name = Utf8View("EUROPE") -20)--------------TableScan: region projection=[r_regionkey, r_name], partial_filters=[region.r_name = Utf8View("EUROPE")] -21)------SubqueryAlias: __scalar_sq_1 -22)--------Projection: min(partsupp.ps_supplycost), partsupp.ps_partkey -23)----------Aggregate: groupBy=[[partsupp.ps_partkey]], aggr=[[min(partsupp.ps_supplycost)]] -24)------------Projection: partsupp.ps_partkey, partsupp.ps_supplycost -25)--------------Inner Join: nation.n_regionkey = region.r_regionkey -26)----------------Projection: partsupp.ps_partkey, partsupp.ps_supplycost, nation.n_regionkey -27)------------------Inner Join: supplier.s_nationkey = nation.n_nationkey -28)--------------------Projection: partsupp.ps_partkey, partsupp.ps_supplycost, supplier.s_nationkey -29)----------------------Inner Join: partsupp.ps_suppkey = supplier.s_suppkey -30)------------------------TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_supplycost] -31)------------------------TableScan: supplier projection=[s_suppkey, s_nationkey] -32)--------------------TableScan: nation projection=[n_nationkey, n_regionkey] -33)----------------Projection: region.r_regionkey -34)------------------Filter: region.r_name = Utf8View("EUROPE") -35)--------------------TableScan: region projection=[r_regionkey, r_name], partial_filters=[region.r_name = Utf8View("EUROPE")] +13)------------------------TableScan: part projection=[p_partkey, p_mfgr, p_type, p_size], partial_filters=[part.p_size = Int32(15), part.p_type LIKE Utf8View("%BRASS")] +14)----------------------TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_supplycost] +15)------------------TableScan: supplier projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment] +16)--------------TableScan: nation projection=[n_nationkey, n_name, n_regionkey] +17)----------Projection: region.r_regionkey +18)------------TableScan: region projection=[r_regionkey, r_name], partial_filters=[region.r_name = Utf8View("EUROPE")] +19)------SubqueryAlias: __scalar_sq_1 +20)--------Projection: min(partsupp.ps_supplycost), partsupp.ps_partkey +21)----------Aggregate: groupBy=[[partsupp.ps_partkey]], aggr=[[min(partsupp.ps_supplycost)]] +22)------------Projection: partsupp.ps_partkey, partsupp.ps_supplycost +23)--------------Inner Join: nation.n_regionkey = region.r_regionkey +24)----------------Projection: partsupp.ps_partkey, partsupp.ps_supplycost, nation.n_regionkey +25)------------------Inner Join: supplier.s_nationkey = nation.n_nationkey +26)--------------------Projection: partsupp.ps_partkey, partsupp.ps_supplycost, supplier.s_nationkey +27)----------------------Inner Join: partsupp.ps_suppkey = supplier.s_suppkey +28)------------------------TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_supplycost] +29)------------------------TableScan: supplier projection=[s_suppkey, s_nationkey] +30)--------------------TableScan: nation projection=[n_nationkey, n_regionkey] +31)----------------Projection: region.r_regionkey +32)------------------TableScan: region projection=[r_regionkey, r_name], partial_filters=[region.r_name = Utf8View("EUROPE")] physical_plan 01)SortPreservingMergeExec: [s_acctbal@0 DESC, n_name@2 ASC NULLS LAST, s_name@1 ASC NULLS LAST, p_partkey@3 ASC NULLS LAST], fetch=10 02)--SortExec: TopK(fetch=10), expr=[s_acctbal@0 DESC, n_name@2 ASC NULLS LAST, s_name@1 ASC NULLS LAST, p_partkey@3 ASC NULLS LAST], preserve_partitioning=[true] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part index 426a1cbaa4e22..492349cd82c90 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part @@ -63,23 +63,20 @@ logical_plan 05)--------Inner Join: supplier.s_nationkey = nation.n_nationkey 06)----------TableScan: supplier projection=[s_suppkey, s_name, s_address, s_nationkey] 07)----------Projection: nation.n_nationkey -08)------------Filter: nation.n_name = Utf8View("CANADA") -09)--------------TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8View("CANADA")] -10)------SubqueryAlias: __correlated_sq_2 -11)--------Projection: partsupp.ps_suppkey -12)----------Inner Join: partsupp.ps_partkey = __scalar_sq_3.l_partkey, partsupp.ps_suppkey = __scalar_sq_3.l_suppkey Filter: CAST(partsupp.ps_availqty AS Float64) > __scalar_sq_3.Float64(0.5) * sum(lineitem.l_quantity) -13)------------LeftSemi Join: partsupp.ps_partkey = __correlated_sq_1.p_partkey -14)--------------TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_availqty] -15)--------------SubqueryAlias: __correlated_sq_1 -16)----------------Projection: part.p_partkey -17)------------------Filter: part.p_name LIKE Utf8View("forest%") -18)--------------------TableScan: part projection=[p_partkey, p_name], partial_filters=[part.p_name LIKE Utf8View("forest%")] -19)------------SubqueryAlias: __scalar_sq_3 -20)--------------Projection: Float64(0.5) * CAST(sum(lineitem.l_quantity) AS Float64), lineitem.l_partkey, lineitem.l_suppkey -21)----------------Aggregate: groupBy=[[lineitem.l_partkey, lineitem.l_suppkey]], aggr=[[sum(lineitem.l_quantity)]] -22)------------------Projection: lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity -23)--------------------Filter: lineitem.l_shipdate >= Date32("1994-01-01") AND lineitem.l_shipdate < Date32("1995-01-01") -24)----------------------TableScan: lineitem projection=[l_partkey, l_suppkey, l_quantity, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("1994-01-01"), lineitem.l_shipdate < Date32("1995-01-01")] +08)------------TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8View("CANADA")] +09)------SubqueryAlias: __correlated_sq_2 +10)--------Projection: partsupp.ps_suppkey +11)----------Inner Join: partsupp.ps_partkey = __scalar_sq_3.l_partkey, partsupp.ps_suppkey = __scalar_sq_3.l_suppkey Filter: CAST(partsupp.ps_availqty AS Float64) > __scalar_sq_3.Float64(0.5) * sum(lineitem.l_quantity) +12)------------LeftSemi Join: partsupp.ps_partkey = __correlated_sq_1.p_partkey +13)--------------TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_availqty] +14)--------------SubqueryAlias: __correlated_sq_1 +15)----------------Projection: part.p_partkey +16)------------------TableScan: part projection=[p_partkey, p_name], partial_filters=[part.p_name LIKE Utf8View("forest%")] +17)------------SubqueryAlias: __scalar_sq_3 +18)--------------Projection: Float64(0.5) * CAST(sum(lineitem.l_quantity) AS Float64), lineitem.l_partkey, lineitem.l_suppkey +19)----------------Aggregate: groupBy=[[lineitem.l_partkey, lineitem.l_suppkey]], aggr=[[sum(lineitem.l_quantity)]] +20)------------------Projection: lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity +21)--------------------TableScan: lineitem projection=[l_partkey, l_suppkey, l_quantity, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("1994-01-01"), lineitem.l_shipdate < Date32("1995-01-01")] physical_plan 01)SortPreservingMergeExec: [s_name@0 ASC NULLS LAST] 02)--SortExec: expr=[s_name@0 ASC NULLS LAST], preserve_partitioning=[true] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part index 5e9192d677532..02a5e284c424e 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part @@ -73,22 +73,18 @@ logical_plan 13)------------------------TableScan: supplier projection=[s_suppkey, s_name, s_nationkey] 14)------------------------SubqueryAlias: l1 15)--------------------------Projection: lineitem.l_orderkey, lineitem.l_suppkey -16)----------------------------Filter: lineitem.l_receiptdate > lineitem.l_commitdate -17)------------------------------TableScan: lineitem projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], partial_filters=[lineitem.l_receiptdate > lineitem.l_commitdate] -18)--------------------Projection: orders.o_orderkey -19)----------------------Filter: orders.o_orderstatus = Utf8View("F") -20)------------------------TableScan: orders projection=[o_orderkey, o_orderstatus], partial_filters=[orders.o_orderstatus = Utf8View("F")] -21)----------------Projection: nation.n_nationkey -22)------------------Filter: nation.n_name = Utf8View("SAUDI ARABIA") -23)--------------------TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8View("SAUDI ARABIA")] -24)------------SubqueryAlias: __correlated_sq_1 -25)--------------SubqueryAlias: l2 -26)----------------TableScan: lineitem projection=[l_orderkey, l_suppkey] -27)----------SubqueryAlias: __correlated_sq_2 -28)------------SubqueryAlias: l3 -29)--------------Projection: lineitem.l_orderkey, lineitem.l_suppkey -30)----------------Filter: lineitem.l_receiptdate > lineitem.l_commitdate -31)------------------TableScan: lineitem projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], partial_filters=[lineitem.l_receiptdate > lineitem.l_commitdate] +16)----------------------------TableScan: lineitem projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], partial_filters=[lineitem.l_receiptdate > lineitem.l_commitdate] +17)--------------------Projection: orders.o_orderkey +18)----------------------TableScan: orders projection=[o_orderkey, o_orderstatus], partial_filters=[orders.o_orderstatus = Utf8View("F")] +19)----------------Projection: nation.n_nationkey +20)------------------TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8View("SAUDI ARABIA")] +21)------------SubqueryAlias: __correlated_sq_1 +22)--------------SubqueryAlias: l2 +23)----------------TableScan: lineitem projection=[l_orderkey, l_suppkey] +24)----------SubqueryAlias: __correlated_sq_2 +25)------------SubqueryAlias: l3 +26)--------------Projection: lineitem.l_orderkey, lineitem.l_suppkey +27)----------------TableScan: lineitem projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], partial_filters=[lineitem.l_receiptdate > lineitem.l_commitdate] physical_plan 01)SortPreservingMergeExec: [numwait@1 DESC, s_name@0 ASC NULLS LAST] 02)--SortExec: expr=[numwait@1 DESC, s_name@0 ASC NULLS LAST], preserve_partitioning=[true] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part index add578c3b079d..25a58b72d7b4e 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part @@ -64,15 +64,13 @@ logical_plan 06)----------Inner Join: Filter: CAST(customer.c_acctbal AS Decimal128(19, 6)) > __scalar_sq_2.avg(customer.c_acctbal) 07)------------Projection: customer.c_phone, customer.c_acctbal 08)--------------LeftAnti Join: customer.c_custkey = __correlated_sq_1.o_custkey -09)----------------Filter: substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8View("13"), Utf8View("31"), Utf8View("23"), Utf8View("29"), Utf8View("30"), Utf8View("18"), Utf8View("17")]) -10)------------------TableScan: customer projection=[c_custkey, c_phone, c_acctbal], partial_filters=[substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8View("13"), Utf8View("31"), Utf8View("23"), Utf8View("29"), Utf8View("30"), Utf8View("18"), Utf8View("17")]), Boolean(true)] -11)----------------SubqueryAlias: __correlated_sq_1 -12)------------------TableScan: orders projection=[o_custkey] -13)------------SubqueryAlias: __scalar_sq_2 -14)--------------Aggregate: groupBy=[[]], aggr=[[avg(customer.c_acctbal)]] -15)----------------Projection: customer.c_acctbal -16)------------------Filter: customer.c_acctbal > Decimal128(Some(0),15,2) AND substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8View("13"), Utf8View("31"), Utf8View("23"), Utf8View("29"), Utf8View("30"), Utf8View("18"), Utf8View("17")]) -17)--------------------TableScan: customer projection=[c_phone, c_acctbal], partial_filters=[customer.c_acctbal > Decimal128(Some(0),15,2), substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8View("13"), Utf8View("31"), Utf8View("23"), Utf8View("29"), Utf8View("30"), Utf8View("18"), Utf8View("17")])] +09)----------------TableScan: customer projection=[c_custkey, c_phone, c_acctbal], partial_filters=[substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8View("13"), Utf8View("31"), Utf8View("23"), Utf8View("29"), Utf8View("30"), Utf8View("18"), Utf8View("17")]), Boolean(true)] +10)----------------SubqueryAlias: __correlated_sq_1 +11)------------------TableScan: orders projection=[o_custkey] +12)------------SubqueryAlias: __scalar_sq_2 +13)--------------Aggregate: groupBy=[[]], aggr=[[avg(customer.c_acctbal)]] +14)----------------Projection: customer.c_acctbal +15)------------------TableScan: customer projection=[c_acctbal, c_phone], partial_filters=[customer.c_acctbal > Decimal128(Some(0),15,2), substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8View("13"), Utf8View("31"), Utf8View("23"), Utf8View("29"), Utf8View("30"), Utf8View("18"), Utf8View("17")])] physical_plan 01)SortPreservingMergeExec: [cntrycode@0 ASC NULLS LAST] 02)--SortExec: expr=[cntrycode@0 ASC NULLS LAST], preserve_partitioning=[true] @@ -87,7 +85,7 @@ physical_plan 11)--------------------CoalescePartitionsExec 12)----------------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(c_custkey@0, o_custkey@0)], projection=[c_phone@1, c_acctbal@2] 13)------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -14)--------------------------FilterExec: substr(c_phone@1, 1, 2) IN (SET) ([13, 31, 23, 29, 30, 18, 17]) +14)--------------------------FilterExec: substr(c_phone@1, 1, 2) IN (SET) ([13, 31, 23, 29, 30, 18, 17]) AND true 15)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 16)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_phone, c_acctbal], file_type=csv, has_header=false 17)------------------------RepartitionExec: partitioning=Hash([o_custkey@0], 4), input_partitions=4 @@ -95,6 +93,6 @@ physical_plan 19)------------------AggregateExec: mode=Final, gby=[], aggr=[avg(customer.c_acctbal)] 20)--------------------CoalescePartitionsExec 21)----------------------AggregateExec: mode=Partial, gby=[], aggr=[avg(customer.c_acctbal)] -22)------------------------FilterExec: c_acctbal@1 > Some(0),15,2 AND substr(c_phone@0, 1, 2) IN (SET) ([13, 31, 23, 29, 30, 18, 17]), projection=[c_acctbal@1] +22)------------------------FilterExec: c_acctbal@0 > Some(0),15,2 AND substr(c_phone@1, 1, 2) IN (SET) ([13, 31, 23, 29, 30, 18, 17]), projection=[c_acctbal@0] 23)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -24)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_phone, c_acctbal], file_type=csv, has_header=false +24)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_acctbal, c_phone], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part index 7fec4e5f5d624..1a225b0b759c9 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part @@ -50,13 +50,10 @@ logical_plan 06)----------Projection: orders.o_orderkey, orders.o_orderdate, orders.o_shippriority 07)------------Inner Join: customer.c_custkey = orders.o_custkey 08)--------------Projection: customer.c_custkey -09)----------------Filter: customer.c_mktsegment = Utf8View("BUILDING") -10)------------------TableScan: customer projection=[c_custkey, c_mktsegment], partial_filters=[customer.c_mktsegment = Utf8View("BUILDING")] -11)--------------Filter: orders.o_orderdate < Date32("1995-03-15") -12)----------------TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority], partial_filters=[orders.o_orderdate < Date32("1995-03-15")] -13)----------Projection: lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount -14)------------Filter: lineitem.l_shipdate > Date32("1995-03-15") -15)--------------TableScan: lineitem projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate > Date32("1995-03-15")] +09)----------------TableScan: customer projection=[c_custkey, c_mktsegment], partial_filters=[customer.c_mktsegment = Utf8View("BUILDING")] +10)--------------TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority], partial_filters=[orders.o_orderdate < Date32("1995-03-15")] +11)----------Projection: lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount +12)------------TableScan: lineitem projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate > Date32("1995-03-15")] physical_plan 01)SortPreservingMergeExec: [revenue@1 DESC, o_orderdate@2 ASC NULLS LAST], fetch=10 02)--SortExec: TopK(fetch=10), expr=[revenue@1 DESC, o_orderdate@2 ASC NULLS LAST], preserve_partitioning=[true] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part index 0007666f15365..ee1bce2fd1774 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part @@ -46,12 +46,10 @@ logical_plan 04)------Projection: orders.o_orderpriority 05)--------LeftSemi Join: orders.o_orderkey = __correlated_sq_1.l_orderkey 06)----------Projection: orders.o_orderkey, orders.o_orderpriority -07)------------Filter: orders.o_orderdate >= Date32("1993-07-01") AND orders.o_orderdate < Date32("1993-10-01") -08)--------------TableScan: orders projection=[o_orderkey, o_orderdate, o_orderpriority], partial_filters=[orders.o_orderdate >= Date32("1993-07-01"), orders.o_orderdate < Date32("1993-10-01")] -09)----------SubqueryAlias: __correlated_sq_1 -10)------------Projection: lineitem.l_orderkey -11)--------------Filter: lineitem.l_receiptdate > lineitem.l_commitdate -12)----------------TableScan: lineitem projection=[l_orderkey, l_commitdate, l_receiptdate], partial_filters=[lineitem.l_receiptdate > lineitem.l_commitdate] +07)------------TableScan: orders projection=[o_orderkey, o_orderpriority, o_orderdate], partial_filters=[orders.o_orderdate >= Date32("1993-07-01"), orders.o_orderdate < Date32("1993-10-01")] +08)----------SubqueryAlias: __correlated_sq_1 +09)------------Projection: lineitem.l_orderkey +10)--------------TableScan: lineitem projection=[l_orderkey, l_commitdate, l_receiptdate], partial_filters=[lineitem.l_receiptdate > lineitem.l_commitdate] physical_plan 01)SortPreservingMergeExec: [o_orderpriority@0 ASC NULLS LAST] 02)--SortExec: expr=[o_orderpriority@0 ASC NULLS LAST], preserve_partitioning=[true] @@ -61,8 +59,8 @@ physical_plan 06)----------AggregateExec: mode=Partial, gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(Int64(1))] 07)------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderpriority@1] 08)--------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -09)----------------FilterExec: o_orderdate@1 >= 1993-07-01 AND o_orderdate@1 < 1993-10-01, projection=[o_orderkey@0, o_orderpriority@2] -10)------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderdate, o_orderpriority], file_type=csv, has_header=false +09)----------------FilterExec: o_orderdate@2 >= 1993-07-01 AND o_orderdate@2 < 1993-10-01, projection=[o_orderkey@0, o_orderpriority@1] +10)------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderpriority, o_orderdate], file_type=csv, has_header=false 11)--------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 12)----------------FilterExec: l_receiptdate@2 > l_commitdate@1, projection=[l_orderkey@0] 13)------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_commitdate, l_receiptdate], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part index d854001f3cc4c..eb1f3fbc2eeb7 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part @@ -58,14 +58,12 @@ logical_plan 13)------------------------Inner Join: customer.c_custkey = orders.o_custkey 14)--------------------------TableScan: customer projection=[c_custkey, c_nationkey] 15)--------------------------Projection: orders.o_orderkey, orders.o_custkey -16)----------------------------Filter: orders.o_orderdate >= Date32("1994-01-01") AND orders.o_orderdate < Date32("1995-01-01") -17)------------------------------TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate], partial_filters=[orders.o_orderdate >= Date32("1994-01-01"), orders.o_orderdate < Date32("1995-01-01")] -18)----------------------TableScan: lineitem projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount] -19)------------------TableScan: supplier projection=[s_suppkey, s_nationkey] -20)--------------TableScan: nation projection=[n_nationkey, n_name, n_regionkey] -21)----------Projection: region.r_regionkey -22)------------Filter: region.r_name = Utf8View("ASIA") -23)--------------TableScan: region projection=[r_regionkey, r_name], partial_filters=[region.r_name = Utf8View("ASIA")] +16)----------------------------TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate], partial_filters=[orders.o_orderdate >= Date32("1994-01-01"), orders.o_orderdate < Date32("1995-01-01")] +17)----------------------TableScan: lineitem projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount] +18)------------------TableScan: supplier projection=[s_suppkey, s_nationkey] +19)--------------TableScan: nation projection=[n_nationkey, n_name, n_regionkey] +20)----------Projection: region.r_regionkey +21)------------TableScan: region projection=[r_regionkey, r_name], partial_filters=[region.r_name = Utf8View("ASIA")] physical_plan 01)SortPreservingMergeExec: [revenue@1 DESC] 02)--SortExec: expr=[revenue@1 DESC], preserve_partitioning=[true] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q6.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q6.slt.part index eb9063d691712..a0c735b9cfbfd 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q6.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q6.slt.part @@ -31,12 +31,11 @@ logical_plan 01)Projection: sum(lineitem.l_extendedprice * lineitem.l_discount) AS revenue 02)--Aggregate: groupBy=[[]], aggr=[[sum(lineitem.l_extendedprice * lineitem.l_discount)]] 03)----Projection: lineitem.l_extendedprice, lineitem.l_discount -04)------Filter: lineitem.l_shipdate >= Date32("1994-01-01") AND lineitem.l_shipdate < Date32("1995-01-01") AND lineitem.l_discount >= Decimal128(Some(5),15,2) AND lineitem.l_discount <= Decimal128(Some(7),15,2) AND lineitem.l_quantity < Decimal128(Some(2400),15,2) -05)--------TableScan: lineitem projection=[l_quantity, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("1994-01-01"), lineitem.l_shipdate < Date32("1995-01-01"), lineitem.l_discount >= Decimal128(Some(5),15,2), lineitem.l_discount <= Decimal128(Some(7),15,2), lineitem.l_quantity < Decimal128(Some(2400),15,2)] +04)------TableScan: lineitem projection=[l_extendedprice, l_discount, l_quantity, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("1994-01-01"), lineitem.l_shipdate < Date32("1995-01-01"), lineitem.l_discount >= Decimal128(Some(5),15,2), lineitem.l_discount <= Decimal128(Some(7),15,2), lineitem.l_quantity < Decimal128(Some(2400),15,2)] physical_plan 01)ProjectionExec: expr=[sum(lineitem.l_extendedprice * lineitem.l_discount)@0 as revenue] 02)--AggregateExec: mode=Final, gby=[], aggr=[sum(lineitem.l_extendedprice * lineitem.l_discount)] 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice * lineitem.l_discount)] -05)--------FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01 AND l_discount@2 >= Some(5),15,2 AND l_discount@2 <= Some(7),15,2 AND l_quantity@0 < Some(2400),15,2, projection=[l_extendedprice@1, l_discount@2] -06)----------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_quantity, l_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false +05)--------FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01 AND l_discount@1 >= Some(5),15,2 AND l_discount@1 <= Some(7),15,2 AND l_quantity@2 < Some(2400),15,2, projection=[l_extendedprice@0, l_discount@1] +06)----------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_extendedprice, l_discount, l_quantity, l_shipdate], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part index b4e70993396e6..3b31f7b0e2097 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part @@ -73,16 +73,13 @@ logical_plan 13)------------------------Projection: supplier.s_nationkey, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate 14)--------------------------Inner Join: supplier.s_suppkey = lineitem.l_suppkey 15)----------------------------TableScan: supplier projection=[s_suppkey, s_nationkey] -16)----------------------------Filter: lineitem.l_shipdate >= Date32("1995-01-01") AND lineitem.l_shipdate <= Date32("1996-12-31") -17)------------------------------TableScan: lineitem projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("1995-01-01"), lineitem.l_shipdate <= Date32("1996-12-31")] -18)------------------------TableScan: orders projection=[o_orderkey, o_custkey] -19)--------------------TableScan: customer projection=[c_custkey, c_nationkey] -20)----------------SubqueryAlias: n1 -21)------------------Filter: nation.n_name = Utf8View("FRANCE") OR nation.n_name = Utf8View("GERMANY") -22)--------------------TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8View("FRANCE") OR nation.n_name = Utf8View("GERMANY")] -23)------------SubqueryAlias: n2 -24)--------------Filter: nation.n_name = Utf8View("GERMANY") OR nation.n_name = Utf8View("FRANCE") -25)----------------TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8View("GERMANY") OR nation.n_name = Utf8View("FRANCE")] +16)----------------------------TableScan: lineitem projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("1995-01-01"), lineitem.l_shipdate <= Date32("1996-12-31")] +17)------------------------TableScan: orders projection=[o_orderkey, o_custkey] +18)--------------------TableScan: customer projection=[c_custkey, c_nationkey] +19)----------------SubqueryAlias: n1 +20)------------------TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8View("FRANCE") OR nation.n_name = Utf8View("GERMANY")] +21)------------SubqueryAlias: n2 +22)--------------TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8View("GERMANY") OR nation.n_name = Utf8View("FRANCE")] physical_plan 01)SortPreservingMergeExec: [supp_nation@0 ASC NULLS LAST, cust_nation@1 ASC NULLS LAST, l_year@2 ASC NULLS LAST] 02)--SortExec: expr=[supp_nation@0 ASC NULLS LAST, cust_nation@1 ASC NULLS LAST, l_year@2 ASC NULLS LAST], preserve_partitioning=[true] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part index 12f19d43d40e7..3ec917f8e5a33 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part @@ -75,20 +75,17 @@ logical_plan 17)--------------------------------Projection: lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount 18)----------------------------------Inner Join: part.p_partkey = lineitem.l_partkey 19)------------------------------------Projection: part.p_partkey -20)--------------------------------------Filter: part.p_type = Utf8View("ECONOMY ANODIZED STEEL") -21)----------------------------------------TableScan: part projection=[p_partkey, p_type], partial_filters=[part.p_type = Utf8View("ECONOMY ANODIZED STEEL")] -22)------------------------------------TableScan: lineitem projection=[l_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount] -23)--------------------------------TableScan: supplier projection=[s_suppkey, s_nationkey] -24)----------------------------Filter: orders.o_orderdate >= Date32("1995-01-01") AND orders.o_orderdate <= Date32("1996-12-31") -25)------------------------------TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate], partial_filters=[orders.o_orderdate >= Date32("1995-01-01"), orders.o_orderdate <= Date32("1996-12-31")] -26)------------------------TableScan: customer projection=[c_custkey, c_nationkey] -27)--------------------SubqueryAlias: n1 -28)----------------------TableScan: nation projection=[n_nationkey, n_regionkey] -29)----------------SubqueryAlias: n2 -30)------------------TableScan: nation projection=[n_nationkey, n_name] -31)------------Projection: region.r_regionkey -32)--------------Filter: region.r_name = Utf8View("AMERICA") -33)----------------TableScan: region projection=[r_regionkey, r_name], partial_filters=[region.r_name = Utf8View("AMERICA")] +20)--------------------------------------TableScan: part projection=[p_partkey, p_type], partial_filters=[part.p_type = Utf8View("ECONOMY ANODIZED STEEL")] +21)------------------------------------TableScan: lineitem projection=[l_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount] +22)--------------------------------TableScan: supplier projection=[s_suppkey, s_nationkey] +23)----------------------------TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate], partial_filters=[orders.o_orderdate >= Date32("1995-01-01"), orders.o_orderdate <= Date32("1996-12-31")] +24)------------------------TableScan: customer projection=[c_custkey, c_nationkey] +25)--------------------SubqueryAlias: n1 +26)----------------------TableScan: nation projection=[n_nationkey, n_regionkey] +27)----------------SubqueryAlias: n2 +28)------------------TableScan: nation projection=[n_nationkey, n_name] +29)------------Projection: region.r_regionkey +30)--------------TableScan: region projection=[r_regionkey, r_name], partial_filters=[region.r_name = Utf8View("AMERICA")] physical_plan 01)SortPreservingMergeExec: [o_year@0 ASC NULLS LAST] 02)--SortExec: expr=[o_year@0 ASC NULLS LAST], preserve_partitioning=[true] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part index 4ec434c90368f..656c62f5e0dfb 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part @@ -67,13 +67,12 @@ logical_plan 13)------------------------Projection: lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount 14)--------------------------Inner Join: part.p_partkey = lineitem.l_partkey 15)----------------------------Projection: part.p_partkey -16)------------------------------Filter: part.p_name LIKE Utf8View("%green%") -17)--------------------------------TableScan: part projection=[p_partkey, p_name], partial_filters=[part.p_name LIKE Utf8View("%green%")] -18)----------------------------TableScan: lineitem projection=[l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount] -19)------------------------TableScan: supplier projection=[s_suppkey, s_nationkey] -20)--------------------TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_supplycost] -21)----------------TableScan: orders projection=[o_orderkey, o_orderdate] -22)------------TableScan: nation projection=[n_nationkey, n_name] +16)------------------------------TableScan: part projection=[p_partkey, p_name], partial_filters=[part.p_name LIKE Utf8View("%green%")] +17)----------------------------TableScan: lineitem projection=[l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount] +18)------------------------TableScan: supplier projection=[s_suppkey, s_nationkey] +19)--------------------TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_supplycost] +20)----------------TableScan: orders projection=[o_orderkey, o_orderdate] +21)------------TableScan: nation projection=[n_nationkey, n_name] physical_plan 01)SortPreservingMergeExec: [nation@0 ASC NULLS LAST, o_year@1 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[nation@0 ASC NULLS LAST, o_year@1 DESC], preserve_partitioning=[true] diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index d858d0ae3ea4e..db1f0a09ccf6c 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -479,17 +479,16 @@ logical_plan 07)------------Projection: 08)--------------Aggregate: groupBy=[[aggregate_test_100.c1]], aggr=[[]] 09)----------------Projection: aggregate_test_100.c1 -10)------------------Filter: aggregate_test_100.c13 != Utf8View("C2GT5KVyOPZpgKVl110TyZO0NcJ434") -11)--------------------TableScan: aggregate_test_100 projection=[c1, c13], partial_filters=[aggregate_test_100.c13 != Utf8View("C2GT5KVyOPZpgKVl110TyZO0NcJ434")] -12)----Projection: Int64(1) AS cnt -13)------Limit: skip=0, fetch=3 -14)--------EmptyRelation: rows=1 -15)----Projection: lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS cnt -16)------Limit: skip=0, fetch=3 -17)--------WindowAggr: windowExpr=[[lead(b.c1, Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] -18)----------SubqueryAlias: b -19)------------Projection: Int64(1) AS c1 -20)--------------EmptyRelation: rows=1 +10)------------------TableScan: aggregate_test_100 projection=[c1, c13], partial_filters=[aggregate_test_100.c13 != Utf8View("C2GT5KVyOPZpgKVl110TyZO0NcJ434")] +11)----Projection: Int64(1) AS cnt +12)------Limit: skip=0, fetch=3 +13)--------EmptyRelation: rows=1 +14)----Projection: lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS cnt +15)------Limit: skip=0, fetch=3 +16)--------WindowAggr: windowExpr=[[lead(b.c1, Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] +17)----------SubqueryAlias: b +18)------------Projection: Int64(1) AS c1 +19)--------------EmptyRelation: rows=1 physical_plan 01)CoalescePartitionsExec: fetch=3 02)--UnionExec @@ -816,10 +815,8 @@ ORDER BY c1 logical_plan 01)Sort: c1 ASC NULLS LAST 02)--Union -03)----Filter: aggregate_test_100.c1 = Utf8View("a") -04)------TableScan: aggregate_test_100 projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], partial_filters=[aggregate_test_100.c1 = Utf8View("a")] -05)----Filter: aggregate_test_100.c1 = Utf8View("a") -06)------TableScan: aggregate_test_100 projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], partial_filters=[aggregate_test_100.c1 = Utf8View("a")] +03)----TableScan: aggregate_test_100 projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], partial_filters=[aggregate_test_100.c1 = Utf8View("a")] +04)----TableScan: aggregate_test_100 projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], partial_filters=[aggregate_test_100.c1 = Utf8View("a")] physical_plan 01)CoalescePartitionsExec 02)--UnionExec diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 8ac8724683a8a..b569e1ec08872 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -1869,8 +1869,7 @@ logical_plan 04)------Projection: 05)--------Aggregate: groupBy=[[aggregate_test_100.c1]], aggr=[[]] 06)----------Projection: aggregate_test_100.c1 -07)------------Filter: aggregate_test_100.c13 != Utf8View("C2GT5KVyOPZpgKVl110TyZO0NcJ434") -08)--------------TableScan: aggregate_test_100 projection=[c1, c13], partial_filters=[aggregate_test_100.c13 != Utf8View("C2GT5KVyOPZpgKVl110TyZO0NcJ434")] +07)------------TableScan: aggregate_test_100 projection=[c1, c13], partial_filters=[aggregate_test_100.c13 != Utf8View("C2GT5KVyOPZpgKVl110TyZO0NcJ434")] physical_plan 01)ProjectionExec: expr=[count(Int64(1))@0 as global_count] 02)--AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] @@ -3502,8 +3501,7 @@ where b=0 ---- logical_plan 01)WindowAggr: windowExpr=[[sum(CAST(multiple_ordered_table.a AS Int64)) ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -02)--Filter: multiple_ordered_table.b = Int32(0) -03)----TableScan: multiple_ordered_table projection=[a0, a, b, c, d], partial_filters=[multiple_ordered_table.b = Int32(0)] +02)--TableScan: multiple_ordered_table projection=[a0, a, b, c, d], partial_filters=[multiple_ordered_table.b = Int32(0)] physical_plan 01)BoundedWindowAggExec: wdw=[sum(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "sum(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": nullable Int64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] 02)--FilterExec: b@2 = 0 @@ -3519,8 +3517,7 @@ where b=0 ---- logical_plan 01)WindowAggr: windowExpr=[[sum(CAST(multiple_ordered_table.a AS Int64)) ORDER BY [multiple_ordered_table.b ASC NULLS LAST, multiple_ordered_table.d ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -02)--Filter: multiple_ordered_table.b = Int32(0) -03)----TableScan: multiple_ordered_table projection=[a0, a, b, c, d], partial_filters=[multiple_ordered_table.b = Int32(0)] +02)--TableScan: multiple_ordered_table projection=[a0, a, b, c, d], partial_filters=[multiple_ordered_table.b = Int32(0)] physical_plan 01)BoundedWindowAggExec: wdw=[sum(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.b ASC NULLS LAST, multiple_ordered_table.d ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "sum(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.b ASC NULLS LAST, multiple_ordered_table.d ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": nullable Int64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] 02)--SortExec: expr=[d@4 ASC NULLS LAST], preserve_partitioning=[false] @@ -3573,8 +3570,7 @@ FROM( logical_plan 01)Projection: max(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS max_c 02)--WindowAggr: windowExpr=[[max(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -03)----Filter: multiple_ordered_table.d = Int32(0) -04)------TableScan: multiple_ordered_table projection=[c, d], partial_filters=[multiple_ordered_table.d = Int32(0)] +03)----TableScan: multiple_ordered_table projection=[c, d], partial_filters=[multiple_ordered_table.d = Int32(0)] physical_plan 01)ProjectionExec: expr=[max(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as max_c] 02)--BoundedWindowAggExec: wdw=[max(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "max(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": nullable Int32 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] @@ -4186,8 +4182,7 @@ EXPLAIN select count(*) over (partition by a order by a) from (select * from a w logical_plan 01)Projection: count(Int64(1)) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS count(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW 02)--WindowAggr: windowExpr=[[count(Int64(1)) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -03)----Filter: a.a = Int64(1) -04)------TableScan: a projection=[a] +03)----TableScan: a projection=[a], unsupported_filters=[a.a = Int64(1)] physical_plan 01)ProjectionExec: expr=[count(Int64(1)) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as count(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] 02)--BoundedWindowAggExec: wdw=[count(Int64(1)) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "count(Int64(1)) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": Int64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] @@ -4206,8 +4201,7 @@ EXPLAIN select ROW_NUMBER() over (partition by a) from (select * from a where a logical_plan 01)Projection: row_number() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING 02)--WindowAggr: windowExpr=[[row_number() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] -03)----Filter: a.a = Int64(1) -04)------TableScan: a projection=[a] +03)----TableScan: a projection=[a], unsupported_filters=[a.a = Int64(1)] physical_plan 01)ProjectionExec: expr=[row_number() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as row_number() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING] 02)--BoundedWindowAggExec: wdw=[row_number() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Field { "row_number() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING": UInt64 }, frame: ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], mode=[Sorted] @@ -5271,8 +5265,7 @@ logical_plan 01)Sort: t1.c1 ASC NULLS LAST, t1.c2 ASC NULLS LAST, rank ASC NULLS LAST 02)--Projection: t1.c1, t1.c2, rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS rank 03)----WindowAggr: windowExpr=[[rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -04)------Filter: t1.c1 = Int32(2) OR t1.c1 = Int32(3) -05)--------TableScan: t1 projection=[c1, c2] +04)------TableScan: t1 projection=[c1, c2], unsupported_filters=[t1.c1 = Int32(2) OR t1.c1 = Int32(3)] physical_plan 01)SortPreservingMergeExec: [c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST, rank@2 ASC NULLS LAST] 02)--ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as rank] @@ -5350,8 +5343,7 @@ logical_plan 02)--Projection: t1.c1, t1.c2, rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS rank 03)----Filter: t1.c2 = Int32(10) 04)------WindowAggr: windowExpr=[[rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -05)--------Filter: t1.c1 = Int32(1) -06)----------TableScan: t1 projection=[c1, c2] +05)--------TableScan: t1 projection=[c1, c2], unsupported_filters=[t1.c1 = Int32(1)] physical_plan 01)SortPreservingMergeExec: [c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST, rank@2 ASC NULLS LAST] 02)--ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as rank] @@ -5429,8 +5421,7 @@ logical_plan 02)--Projection: t1.c1, t1.c2, rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS rank1, rank() PARTITION BY [t1.c2, t1.c1] ORDER BY [t1.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS rank2 03)----WindowAggr: windowExpr=[[rank() PARTITION BY [t1.c2, t1.c1] ORDER BY [t1.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] 04)------WindowAggr: windowExpr=[[rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -05)--------Filter: t1.c1 > Int32(1) -06)----------TableScan: t1 projection=[c1, c2] +05)--------TableScan: t1 projection=[c1, c2], unsupported_filters=[t1.c1 > Int32(1)] physical_plan 01)SortPreservingMergeExec: [c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST, rank1@2 ASC NULLS LAST, rank2@3 ASC NULLS LAST] 02)--SortExec: expr=[c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST, rank1@2 ASC NULLS LAST, rank2@3 ASC NULLS LAST], preserve_partitioning=[true]