From a653b15b924f325450604348d8965c5a64638068 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Thu, 3 Apr 2025 12:45:58 -0500 Subject: [PATCH 01/40] ExecutionPlan: add APIs for filter pushdown & optimizer rule to apply them --- .../physical_optimizer/filter_pushdown.rs | 267 ++++++++++ .../core/tests/physical_optimizer/mod.rs | 1 + .../physical-optimizer/src/filter_pushdown.rs | 501 ++++++++++++++++++ datafusion/physical-optimizer/src/lib.rs | 1 + .../physical-optimizer/src/optimizer.rs | 5 + .../physical-plan/src/execution_plan.rs | 98 ++++ 6 files changed, 873 insertions(+) create mode 100644 datafusion/core/tests/physical_optimizer/filter_pushdown.rs create mode 100644 datafusion/physical-optimizer/src/filter_pushdown.rs diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs new file mode 100644 index 0000000000000..6c6cb0f20af38 --- /dev/null +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs @@ -0,0 +1,267 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use arrow_schema::SortOptions; +use datafusion::{ + datasource::object_store::ObjectStoreUrl, + logical_expr::Operator, + physical_plan::{ + expressions::{BinaryExpr, Column, Literal}, + PhysicalExpr, + }, + scalar::ScalarValue, +}; +use datafusion_common::config::{ConfigOptions, TableParquetOptions}; +use datafusion_common::internal_err; +use datafusion_datasource::file::FileSource; +use datafusion_datasource::file_scan_config::FileScanConfigBuilder; +use datafusion_datasource::source::DataSourceExec; +use datafusion_datasource_parquet::source::ParquetSource; +use datafusion_physical_expr::{LexOrdering, PhysicalSortExpr}; +use datafusion_physical_optimizer::filter_pushdown::PushdownFilter; +use datafusion_physical_optimizer::PhysicalOptimizerRule; +use datafusion_physical_plan::{displayable, ExecutionPlan}; +use datafusion_physical_plan::{filter::FilterExec, sorts::sort::SortExec}; +use std::fmt::{Display, Formatter}; +use std::sync::{Arc, OnceLock}; + +#[test] +fn test_pushdown_into_scan() { + let scan = parquet_scan(); + let predicate = col_lit_predicate("a", "foo", schema()); + let plan = Arc::new(FilterExec::try_new(predicate, scan).unwrap()); + + // expect the predicate to be pushed down into the DataSource + insta::assert_snapshot!( + OptimizationTest::new(plan, PushdownFilter{}), + @r" + OptimizationTest: + input: + - FilterExec: a@0 = foo + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=parquet + output: + Ok: + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=parquet, predicate=a@0 = foo + " + ); +} + +#[test] +fn test_parquet_pushdown() { + // filter should be pushed down into the parquet scan with two filters + let scan = parquet_scan(); + let predicate1 = col_lit_predicate("a", "foo", schema()); + let filter1 = Arc::new(FilterExec::try_new(predicate1, scan).unwrap()); + let predicate2 = col_lit_predicate("b", "bar", schema()); + let plan = Arc::new(FilterExec::try_new(predicate2, filter1).unwrap()); + + insta::assert_snapshot!( + OptimizationTest::new(plan, PushdownFilter{}), + @r" + OptimizationTest: + input: + - FilterExec: b@1 = bar + - FilterExec: a@0 = foo + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=parquet + output: + Ok: + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=parquet, predicate=a@0 = foo AND b@1 = bar + " + ); +} + +#[test] +fn test_topk_pushdown() { + // filter should be pushed down into the parquet scan with two filters + let scan = parquet_scan(); + let predicate = col_lit_predicate("a", "foo", schema()); + let filter = + Arc::new(FilterExec::try_new(Arc::clone(&predicate), Arc::clone(&scan)).unwrap()); + let plan = Arc::new(SortExec::new( + LexOrdering::new(vec![PhysicalSortExpr::new( + Arc::new(Column::new_with_schema("a", schema()).unwrap()), + SortOptions::default(), + )]), + filter, + )); + + insta::assert_snapshot!( + OptimizationTest::new(plan, PushdownFilter{}), + @r" + OptimizationTest: + input: + - SortExec: expr=[a@0 ASC], preserve_partitioning=[false] + - FilterExec: a@0 = foo + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=parquet + output: + Ok: + - SortExec: expr=[a@0 ASC], preserve_partitioning=[false] + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=parquet, predicate=a@0 = foo AND DynamicFilterPhysicalExpr [ SortDynamicFilterSource[ ] ] + " + ); + + let sort = Arc::new(SortExec::new( + LexOrdering::new(vec![PhysicalSortExpr::new( + Arc::new(Column::new_with_schema("a", schema()).unwrap()), + SortOptions::default(), + )]), + Arc::clone(&scan), + )); + let plan = Arc::new(FilterExec::try_new(predicate, sort).unwrap()); + + insta::assert_snapshot!( + OptimizationTest::new(plan, PushdownFilter{}), + @r" + OptimizationTest: + input: + - FilterExec: a@0 = foo + - SortExec: expr=[a@0 ASC], preserve_partitioning=[false] + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=parquet + output: + Ok: + - SortExec: expr=[a@0 ASC], preserve_partitioning=[false] + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ SortDynamicFilterSource[ ] ] AND a@0 = foo + " + ); +} + +/// Schema: +/// a: String +/// b: String +/// c: f64 +static TEST_SCHEMA: OnceLock = OnceLock::new(); + +fn schema() -> &'static SchemaRef { + TEST_SCHEMA.get_or_init(|| { + let fields = vec![ + Field::new("a", DataType::Utf8, false), + Field::new("b", DataType::Utf8, false), + Field::new("c", DataType::Float64, false), + ]; + Arc::new(Schema::new(fields)) + }) +} + +/// Return a execution plan that reads from a parquet file +fn parquet_scan() -> Arc { + let schema = schema(); + let mut options = TableParquetOptions::default(); + options.global.pushdown_filters = true; + let source = ParquetSource::new(options).with_schema(Arc::clone(schema)); + let base_config = FileScanConfigBuilder::new( + ObjectStoreUrl::parse("test://").unwrap(), + Arc::clone(schema), + source, + ) + .build(); + DataSourceExec::from_data_source(base_config) +} + +/// Returns a predicate that is a binary expression col = lit +fn col_lit_predicate( + column_name: &str, + scalar_value: impl Into, + schema: &Schema, +) -> Arc { + let scalar_value = scalar_value.into(); + Arc::new(BinaryExpr::new( + Arc::new(Column::new_with_schema(column_name, schema).unwrap()), + Operator::Eq, + Arc::new(Literal::new(scalar_value)), + )) +} + +/// A harness for testing physical optimizers. +/// +/// You can use this to test the output of a physical optimizer rule using insta snapshots +#[derive(Debug)] +pub struct OptimizationTest { + input: Vec, + output: Result, String>, +} + +impl OptimizationTest { + pub fn new(input_plan: Arc, opt: O) -> Self + where + O: PhysicalOptimizerRule, + { + Self::new_with_config(input_plan, opt, &ConfigOptions::default()) + } + + pub fn new_with_config( + input_plan: Arc, + opt: O, + config: &ConfigOptions, + ) -> Self + where + O: PhysicalOptimizerRule, + { + let input = format_execution_plan(&input_plan); + + let input_schema = input_plan.schema(); + + let output_result = opt.optimize(input_plan, config); + let output = output_result + .and_then(|plan| { + if opt.schema_check() && (plan.schema() != input_schema) { + internal_err!( + "Schema mismatch:\n\nBefore:\n{:?}\n\nAfter:\n{:?}", + input_schema, + plan.schema() + ) + } else { + Ok(plan) + } + }) + .map(|plan| format_execution_plan(&plan)) + .map_err(|e| e.to_string()); + + Self { input, output } + } +} + +impl Display for OptimizationTest { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + writeln!(f, "OptimizationTest:")?; + writeln!(f, " input:")?; + for line in &self.input { + writeln!(f, " - {line}")?; + } + writeln!(f, " output:")?; + match &self.output { + Ok(output) => { + writeln!(f, " Ok:")?; + for line in output { + writeln!(f, " - {line}")?; + } + } + Err(err) => { + writeln!(f, " Err: {err}")?; + } + } + Ok(()) + } +} + +pub fn format_execution_plan(plan: &Arc) -> Vec { + format_lines(&displayable(plan.as_ref()).indent(false).to_string()) +} + +fn format_lines(s: &str) -> Vec { + s.trim().split('\n').map(|s| s.to_string()).collect() +} diff --git a/datafusion/core/tests/physical_optimizer/mod.rs b/datafusion/core/tests/physical_optimizer/mod.rs index 7d5d07715eebc..fe7b9decfebfa 100644 --- a/datafusion/core/tests/physical_optimizer/mod.rs +++ b/datafusion/core/tests/physical_optimizer/mod.rs @@ -21,6 +21,7 @@ mod aggregate_statistics; mod combine_partial_final_agg; mod enforce_distribution; mod enforce_sorting; +mod filter_pushdown; mod join_selection; mod limit_pushdown; mod limited_distinct_aggregation; diff --git a/datafusion/physical-optimizer/src/filter_pushdown.rs b/datafusion/physical-optimizer/src/filter_pushdown.rs new file mode 100644 index 0000000000000..21610564306cd --- /dev/null +++ b/datafusion/physical-optimizer/src/filter_pushdown.rs @@ -0,0 +1,501 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::sync::Arc; + +use datafusion_common::{config::ConfigOptions, Result}; +use datafusion_physical_expr::PhysicalExpr; +use datafusion_physical_plan::{ + execution_plan::{ + ExecutionPlanFilterPushdownResult, FilterPushdownAllowed, FilterSupport, + }, + with_new_children_if_necessary, ExecutionPlan, +}; + +use crate::PhysicalOptimizerRule; + +/// The state of filter pushdown support for a given filter. +#[derive(Clone, Copy, Debug)] +enum ChildPushdownState { + /// A child said it can handle the filter exactly. + ChildExact, + /// A child exists and took a look at the filter. + /// It may partially handle it or not handle it at all. + /// The parent still needs to re-apply the filter. + ChildInexact, + /// No child exists, there is no one to handle the filter. + /// This is the default / initial state. + NoChild, +} + +impl ChildPushdownState { + /// Combine the current state with another state. + /// This is used to combine the results of multiple children. + fn combine_with_other(&self, other: &FilterSupport) -> ChildPushdownState { + match (other, self) { + (FilterSupport::HandledExact, ChildPushdownState::NoChild) => { + ChildPushdownState::ChildExact + } + (FilterSupport::HandledExact, ChildPushdownState::ChildInexact) => { + ChildPushdownState::ChildInexact + } + (FilterSupport::Unhandled, ChildPushdownState::NoChild) => { + ChildPushdownState::ChildInexact + } + (FilterSupport::Unhandled, ChildPushdownState::ChildExact) => { + ChildPushdownState::ChildInexact + } + (FilterSupport::Unhandled, ChildPushdownState::ChildInexact) => { + ChildPushdownState::ChildInexact + } + (FilterSupport::HandledExact, ChildPushdownState::ChildExact) => { + // If both are exact, keep it as exact + ChildPushdownState::ChildExact + } + } + } +} + +/// See [`pushdown_filters`] for more details. +fn push_down_into_children( + node: &Arc, + filters: &[Arc], +) -> Result { + let children = node.children(); + let mut new_children = Vec::with_capacity(children.len()); + let mut filter_pushdown_result = vec![ChildPushdownState::NoChild; filters.len()]; + for child in children { + if let Some(result) = pushdown_filters(child, filters)? { + new_children.push(result.inner); + for (idx, support) in result.support.iter().enumerate() { + filter_pushdown_result[idx] = + filter_pushdown_result[idx].combine_with_other(support) + } + } else { + new_children.push(Arc::clone(child)); + } + } + let support = filter_pushdown_result + .iter() + .map(|s| match s { + ChildPushdownState::ChildExact => FilterSupport::HandledExact, + ChildPushdownState::ChildInexact => FilterSupport::Unhandled, + ChildPushdownState::NoChild => FilterSupport::Unhandled, + }) + .collect::>(); + let node = with_new_children_if_necessary(Arc::clone(node), new_children)?; + Ok(ExecutionPlanFilterPushdownResult::new(node, support)) +} + +/// Recursively a collection of filters down through the execution plan tree in a depth-first manner. +/// +/// For each filter we try to push it down to children as far down as possible, keeping track of if the children +/// can handle the filter or not. +/// +/// If a child can handle the filter, we mark it as handled exact and parent nodes (including the source of the filter) +/// can decide to discard it / not re-apply it themselves. +/// If a child cannot handle the filter or may return false positives (aka "inexact" handling) we mark it as handled inexact. +/// If a child does not allow filter pushdown at all (e.g. an aggregation node) we keep recursing but clear the current set of filters +/// we are pushing down. +/// +/// As we recurse back up the tree we combine the results of the children to determine if the overall result is exact or inexact: +/// - For nodes with a single child we just take the child's result. +/// - For nodes with multiple children we combine the results of the children to determine if the overall result is exact or inexact. +/// We do this by checking if all children are exact (we return exact up) or if any child is inexact (we return inexact). +/// - If a node has no children this is equivalent to inexact handling (there is no child to handle the filter). +/// +/// See [`PushdownFilter`] for more details on how this works in practice. +fn pushdown_filters( + node: &Arc, + parent_filters: &[Arc], +) -> Result> { + // Gather the filters from the current node. + // These are the filters the current node "owns" or "produces" and wants to push down. + let node_filters = node.filters_for_pushdown()?; + // Check which nodes from parents this node is okay with us trying to push down to it's children. + let parent_pushdown_request_result = node.filter_pushdown_request(parent_filters)?; + // Do some index masking so that we only ever call nodes with the filters relevant to them / that they're allowed to touch. + // But we still need to reconstruct the full result for our caller. + let parent_filter_for_pushdown_indices = parent_pushdown_request_result + .iter() + .enumerate() + .filter_map(|(i, s)| { + if matches!(s, FilterPushdownAllowed::Allowed(_)) { + Some(i) + } else { + None + } + }) + .collect::>(); + let parent_filters_to_push_down = parent_filter_for_pushdown_indices + .iter() + .map(|&i| Arc::clone(&parent_filters[i])) + .collect::>(); + let all_filters_to_push_down = node_filters + .iter() + .chain(parent_filters_to_push_down.iter()) + .map(Arc::clone) + .collect::>(); + // Push down into children + let child_pushdown_result = push_down_into_children(node, &all_filters_to_push_down)?; + let mut node = child_pushdown_result.inner; + // A bit more index masking to construct the final result for our caller. + let node_filters_pushdown_result = + child_pushdown_result.support[..node_filters.len()].to_vec(); + let mut parent_filter_pushdown_result = + vec![FilterSupport::Unhandled; parent_filters.len()]; + for (parent_filter_idx, support) in parent_filter_for_pushdown_indices + .iter() + .zip(child_pushdown_result.support[node_filters.len()..].iter()) + { + parent_filter_pushdown_result[*parent_filter_idx] = *support; + } + // Collect the remaining unhandled parent filters + let unhandled_parent_filter_indices = (0..parent_filters.len()) + .filter(|&i| matches!(parent_filter_pushdown_result[i], FilterSupport::Unhandled)) + .collect::>(); + let unhandled_parent_filters = unhandled_parent_filter_indices + .iter() + .map(|&i| Arc::clone(&parent_filters[i])) + .collect::>(); + // Check if the node can handle the filters + if let Some(result) = Arc::clone(&node).with_filter_pushdown_result( + &node_filters_pushdown_result, + &unhandled_parent_filters, + )? { + node = result.inner; + for (parent_filter_index, support) in + unhandled_parent_filter_indices.iter().zip(result.support) + { + parent_filter_pushdown_result[*parent_filter_index] = support; + } + } + Ok(Some(ExecutionPlanFilterPushdownResult::new( + node, + parent_filter_pushdown_result, + ))) +} + +/// A physical optimizer rule that pushes down filters in the execution plan. +/// For example, consider the following plan: +/// +/// ```text +/// ┌──────────────────────┐ +/// │ CoalesceBatchesExec │ +/// └──────────────────────┘ +/// │ +/// ▼ +/// ┌──────────────────────┐ +/// │ FilterExec │ +/// │ filters = [ id=1] │ +/// └──────────────────────┘ +/// │ +/// ▼ +/// ┌──────────────────────┐ +/// │ DataSourceExec │ +/// │ projection = * │ +/// └──────────────────────┘ +/// ``` +/// +/// Our goal is to move the `id = 1` filter from the `FilterExec` node to the `DataSourceExec` node. +/// If this filter is selective it can avoid massive amounts of data being read from the source (the projection is `*` so all matching columns are read). +/// In this simple case we: +/// 1. Enter the recursion with no filters. +/// 2. We find the `FilterExec` node and it tells us that it has a filter (see [`ExecutionPlan::filters_for_pushdown`] and `datafusion::physical_plan::filter::FilterExec`). +/// 3. We recurse down into it's children (the `DataSourceExec` node) now carrying the filters `[id = 1]`. +/// 4. The `DataSourceExec` node tells us that it can handle the filter and we mark it as handled exact (see [`ExecutionPlan::with_filter_pushdown_result`]). +/// 5. Since the `DataSourceExec` node has no children we recurse back up the tree. +/// 6. We now tell the `FilterExec` node that it has a child that can handle the filter and we mark it as handled exact (see [`ExecutionPlan::with_filter_pushdown_result`]). +/// The `FilterExec` node can now return a new execution plan, either a copy of itself without that filter or if has no work left to do it can even return the child node directly. +/// 7. We recurse back up to `CoalesceBatchesExec` and do nothing there since it had no filters to push down. +/// +/// The new plan looks like: +/// +/// ```text +/// ┌──────────────────────┐ +/// │ CoalesceBatchesExec │ +/// └──────────────────────┘ +/// │ +/// ▼ +/// ┌──────────────────────┐ +/// │ DataSourceExec │ +// │ projection = * │ +// │ filters = [ id=1] │ +/// └──────────────────────┘ +/// ``` +/// +/// Let's consider a more complex example involving a `ProjectionExec` node in betweeen the `FilterExec` and `DataSourceExec` nodes that creates a new column that the filter depends on. +/// +/// ```text +// ┌──────────────────────┐ +// │ CoalesceBatchesExec │ +// └──────────────────────┘ +// │ +// ▼ +// ┌──────────────────────┐ +// │ FilterExec │ +// │ filters = │ +// │ [cost>50,id=1] │ +// └──────────────────────┘ +// │ +// ▼ +// ┌──────────────────────┐ +// │ ProjectionExec │ +// │ cost = price * 1.2 │ +// └──────────────────────┘ +// │ +// ▼ +// ┌──────────────────────┐ +// │ DataSourceExec │ +// │ projection = * │ +// └──────────────────────┘ +/// ``` +/// +/// We want to push down the filters [id=1] to the `DataSourceExec` node, but can't push down `cost>50` because it requires the `ProjectionExec` node to be executed first: +/// +/// ```text +// ┌──────────────────────┐ +// │ CoalesceBatchesExec │ +// └──────────────────────┘ +// │ +// ▼ +// ┌──────────────────────┐ +// │ FilterExec │ +// │ filters = │ +// │ [cost>50] │ +// └──────────────────────┘ +// │ +// ▼ +// ┌──────────────────────┐ +// │ ProjectionExec │ +// │ cost = price * 1.2 │ +// └──────────────────────┘ +// │ +// ▼ +// ┌──────────────────────┐ +// │ DataSourceExec │ +// │ projection = * │ +// │ filters = [ id=1] │ +// └──────────────────────┘ +/// ``` +/// +/// There are also cases where we may be able to push down filters within a subtree but not the entire tree. +/// A good exmaple of this is aggreagation nodes: +/// +/// ```text +/// ┌──────────────────────┐ +/// │ ProjectionExec │ +/// │ projection = * │ +/// └──────────────────────┘ +/// │ +/// ▼ +/// ┌──────────────────────┐ +/// │ FilterExec │ +/// │ filters = [sum > 10] │ +/// └──────────────────────┘ +/// │ +/// ▼ +/// ┌───────────────────────┐ +/// │ AggregateExec │ +/// │ group by = [id] │ +/// │ aggregate = │ +/// │ [sum(price)] │ +/// └───────────────────────┘ +/// │ +/// ▼ +/// ┌──────────────────────┐ +/// │ FilterExec │ +/// │ filters = [id=1] │ +/// └──────────────────────┘ +/// │ +/// ▼ +/// ┌──────────────────────┐ +/// │ DataSourceExec │ +/// │ projection = * │ +/// └──────────────────────┘ +/// ``` +/// +/// The transformation here is to push down the `[id=1]` filter to the `DataSourceExec` node: +/// +/// ```text +/// ┌──────────────────────┐ +/// │ ProjectionExec │ +/// │ projection = * │ +/// └──────────────────────┘ +/// │ +/// ▼ +/// ┌──────────────────────┐ +/// │ FilterExec │ +/// │ filters = [sum > 10] │ +/// └──────────────────────┘ +/// │ +/// ▼ +/// ┌───────────────────────┐ +/// │ AggregateExec │ +/// │ group by = [id] │ +/// │ aggregate = │ +/// │ [sum(price)] │ +/// └───────────────────────┘ +/// │ +/// ▼ +/// ┌──────────────────────┐ +/// │ DataSourceExec │ +/// │ projection = * │ +/// │ filters = [id=1] │ +/// └──────────────────────┘ +/// ``` +/// +/// The point here is that: +/// 1. We cannot push down `sum > 10` through the `AggregateExec` node into the `DataSourceExec` node. +/// Any filters above the `AggregateExec` node are not pushed down. +/// This is determined by calling [`ExecutionPlan::filter_pushdown_request`] on the `AggregateExec` node. +/// 2. We need to keep recursing into the tree so that we can discover the other `FilterExec` node and push down the [id=1] filter. +/// +/// It is also possible to push down filters through joins and from joins. +/// For example, a hash join where we build a hash table of the left side and probe the right side +/// (ignoring why we would choose this order, typically it depends on the size of each table, etc.). +/// +/// ```text +/// ┌─────────────────────┐ +/// │ FilterExec │ +/// │ filters = │ +/// │ [d.size > 100] │ +/// └─────────────────────┘ +/// │ +/// │ +/// ┌──────────▼──────────┐ +/// │ │ +/// │ HashJoinExec │ +/// │ [u.dept@hash(d.id)] │ +/// │ │ +/// └─────────────────────┘ +/// │ +/// ┌────────────┴────────────┐ +/// ┌──────────▼──────────┐ ┌──────────▼──────────┐ +/// │ DataSourceExec │ │ DataSourceExec │ +/// │ alias [users as u] │ │ alias [dept as d] │ +/// │ │ │ │ +/// └─────────────────────┘ └─────────────────────┘ +/// ``` +/// +/// There are two pushdowns we can do here: +/// 1. Push down the `d.size > 100` filter through the `HashJoinExec` node to the `DataSourceExec` node for the `departments` table. +/// 2. Push down the hash table state from the `HashJoinExec` node to the `DataSourceExec` node to avoid reading +/// rows from teh `users` table that will be eliminated by the join. +/// This can be done via a bloom filter or similar. +/// +/// ```text +/// ┌─────────────────────┐ +/// │ │ +/// │ HashJoinExec │ +/// │ [u.dept@hash(d.id)] │ +/// │ │ +/// └─────────────────────┘ +/// │ +/// ┌────────────┴────────────┐ +/// ┌──────────▼──────────┐ ┌──────────▼──────────┐ +/// │ DataSourceExec │ │ DataSourceExec │ +/// │ alias [users as u] │ │ alias [dept as d] │ +/// │ filters = │ │ filters = │ +/// │ [depg@hash(d.id)] │ │ [ d.size > 100] │ +/// └─────────────────────┘ └─────────────────────┘ +/// ``` +/// +/// You may notice in this case that the filter is *dynamic*: the hash table is built +/// _after_ the `departments` table is read and at runtime. +/// We don't have a concrete `InList` filter or similar to push down at optimization time. +/// These sorts of dynamic filters are handled by building a specialized [`PhysicalExpr`] that +/// internally maintains a reference to the hash table or other state. +/// To make working with these sorts of dynamic filters more tractable we have the method [`PhysicalExpr::snapshot`] +/// which attempts to simplify a dynamic filter into a "basic" non-dynamic filter. +/// For a join this could mean converting it to an `InList` filter or a min/max filter for example. +/// See `datafusion/physical-plan/src/dynamic_filters.rs` for more details. +/// +/// Another form of dyanmic filter is pushing down the state of a `TopK` operator for queries like +/// `SELECT * FROM t ORDER BY id LIMIT 10`: +/// +/// ```text +/// ┌──────────────────────┐ +/// │ TopK │ +/// │ limit = 10 │ +/// │ order by = [id] │ +/// └──────────────────────┘ +/// │ +/// ▼ +/// ┌──────────────────────┐ +/// │ DataSourceExec │ +/// │ projection = * │ +/// └──────────────────────┘ +/// ``` +/// +/// We can avoid large amounts of data processing by transforming this into: +/// +/// ```text +/// ┌──────────────────────┐ +/// │ TopK │ +/// │ limit = 10 │ +/// │ order by = [id] │ +/// └──────────────────────┘ +/// │ +/// ▼ +/// ┌──────────────────────┐ +/// │ DataSourceExec │ +/// │ projection = * │ +/// │ filters = │ +/// │ [id < @ TopKHeap] │ +/// └──────────────────────┘ +/// ``` +/// +/// Now as we fill our `TopK` heap we can push down the state of the heap to the `DataSourceExec` node +/// to avoid reading files / row groups / pages / rows that could not possibly be in the top 10. +/// This is implemented in datafusion/physical-plan/src/sorts/sort_filters.rs. +#[derive(Debug)] +pub struct PushdownFilter {} + +impl Default for PushdownFilter { + fn default() -> Self { + Self::new() + } +} + +impl PushdownFilter { + pub fn new() -> Self { + Self {} + } +} + +impl PhysicalOptimizerRule for PushdownFilter { + fn optimize( + &self, + plan: Arc, + _config: &ConfigOptions, + ) -> Result> { + if let Some(result) = pushdown_filters(&plan, &[])? { + Ok(result.inner) + } else { + Ok(plan) + } + } + + fn name(&self) -> &str { + "FilterPushdown" + } + + fn schema_check(&self) -> bool { + true // Filter pushdown does not change the schema of the plan + } +} diff --git a/datafusion/physical-optimizer/src/lib.rs b/datafusion/physical-optimizer/src/lib.rs index 35503f3b0b5f9..5a43d7118d638 100644 --- a/datafusion/physical-optimizer/src/lib.rs +++ b/datafusion/physical-optimizer/src/lib.rs @@ -29,6 +29,7 @@ pub mod coalesce_batches; pub mod combine_partial_final_agg; pub mod enforce_distribution; pub mod enforce_sorting; +pub mod filter_pushdown; pub mod join_selection; pub mod limit_pushdown; pub mod limited_distinct_aggregation; diff --git a/datafusion/physical-optimizer/src/optimizer.rs b/datafusion/physical-optimizer/src/optimizer.rs index bab31150e2508..8bd22cbf1bdab 100644 --- a/datafusion/physical-optimizer/src/optimizer.rs +++ b/datafusion/physical-optimizer/src/optimizer.rs @@ -25,6 +25,7 @@ use crate::coalesce_batches::CoalesceBatches; use crate::combine_partial_final_agg::CombinePartialFinalAggregate; use crate::enforce_distribution::EnforceDistribution; use crate::enforce_sorting::EnforceSorting; +use crate::filter_pushdown::PushdownFilter; use crate::join_selection::JoinSelection; use crate::limit_pushdown::LimitPushdown; use crate::limited_distinct_aggregation::LimitedDistinctAggregation; @@ -121,6 +122,10 @@ impl PhysicalOptimizer { // into an `order by max(x) limit y`. In this case it will copy the limit value down // to the aggregation, allowing it to use only y number of accumulators. Arc::new(TopKAggregation::new()), + // The FilterPushdown rule tries to push down filters as far as it can. + // For example, it will push down filtering from a `FilterExec` to + // a `DataSourceExec`, or from a `TopK`'s current state to a `DataSourceExec`. + Arc::new(PushdownFilter::new()), // The LimitPushdown rule tries to push limits down as far as possible, // replacing operators with fetching variants, or adding limits // past operators that support limit pushdown. diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index 2bc5706ee0e18..595e1afd32c6a 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -467,8 +467,106 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { ) -> Result>> { Ok(None) } + + /// Returns a set of filters that this operator owns but would like to be pushed down. + /// For example, a `TopK` operator may produce dynamic filters that reference it's currrent state, + /// while a `FilterExec` will just hand of the filters it has as is. + /// The default implementation returns an empty vector. + /// These filters are applied row-by row and any that return `false` or `NULL` will be + /// filtered out and any that return `true` will be kept. + /// The expressions returned **must** always return `true` or `false`; + /// other truthy or falsy values are not allowed (e.g. `0`, `1`). + /// + /// # Returns + /// A vector of filters that this operator would like to push down. + /// These should be treated as the split conjunction of a `WHERE` clause. + /// That is, a query such as `WHERE a = 1 AND b = 2` would return two + /// filters: `a = 1` and `b = 2`. + /// They can always be assembled into a single filter using + /// [`crate::physical_expr::split_conjunction`]. + fn filters_for_pushdown(&self) -> Result>> { + Ok(Vec::new()) + } + + /// Checks which filters this node allows to be pushed down through it from a parent to a child. + /// For example, a `ProjectionExec` node can allow filters that only refernece + /// columns it did not create through but filters that reference columns it is creating cannot be pushed down any further. + /// That is, it only allows some filters through because it changes the schema of the data. + /// Aggregation nodes may not allow any filters to be pushed down as they change the cardinality of the data. + /// RepartitionExec nodes allow all filters to be pushed down as they don't change the schema or cardinality. + fn filter_pushdown_request( + &self, + filters: &[Arc], + ) -> Result> { + Ok(vec![FilterPushdownAllowed::Disallowed; filters.len()]) + } + + /// After we've attempted to push down filters into this node's children + /// this will be called with the result for each filter that this node gave in `filters_for_pushdown` + /// **and** any filters that children could not handle. + fn with_filter_pushdown_result( + self: Arc, + _own_filters_result: &[FilterSupport], + _parent_filters_remaining: &[Arc], + ) -> Result> { + Ok(None) + } +} + +/// The answer to the question: "Can this filter be pushed down through this plan?" +/// Note that this is different from [`FilterSupport`] which is the answer to "Can *this* plan handle this filter?" +#[derive(Debug, Clone)] +pub enum FilterPushdownAllowed { + /// The operator allows this filter to be pushed down to its children. + /// The operator may choose to return a *different* filter expression + /// that is equivalent to the original filter, e.g. to deal with column indexes in a projection + /// or because the original filter can't be pushed down as is but a less-selective filter can be. + Allowed(Arc), + /// The operator does not allow this filter to be pushed down to its children. + Disallowed, } +/// The answer to the question: "Can this operator handle this filter itself?" +/// Note that this is different from [`FilterPushdownAllowed`] which is the answer to "Can *this* plan handle this filter?" +#[derive(Debug, Clone, Copy)] +pub enum FilterSupport { + /// Filter may not have been pushed down to the child plan, or the child plan + /// can only partially apply the filter but may have false positives (but not false negatives). + /// In this case the parent **must** behave as if the filter was not pushed down + /// and must apply the filter itself. + Unhandled, + /// Filter was pushed down to the child plan and the child plan promises that + /// it will apply the filter correctly with no false positives or false negatives. + /// The parent can safely drop the filter. + HandledExact, +} + +/// The combined result of a filter pushdown operation. +/// This includes: +/// * The inner plan that was produced by the pushdown operation. +/// * The support for each filter that was pushed down. +pub struct FilterPushdownResult { + pub inner: T, + pub support: Vec, +} + +impl FilterPushdownResult { + pub fn new(plan: T, support: Vec) -> Self { + Self { + inner: plan, + support, + } + } + + pub fn is_exact(&self) -> bool { + self.support + .iter() + .all(|s| matches!(s, FilterSupport::HandledExact)) + } +} + +pub type ExecutionPlanFilterPushdownResult = FilterPushdownResult>; + /// [`ExecutionPlan`] Invariant Level /// /// What set of assertions ([Invariant]s) holds for a particular `ExecutionPlan` From a5f998cff68370512d411eeb4f7d5cc1ec5706d2 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Thu, 3 Apr 2025 12:48:36 -0500 Subject: [PATCH 02/40] wip --- .../physical_optimizer/filter_pushdown.rs | 55 ------------------- 1 file changed, 55 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs index 6c6cb0f20af38..ef5bbf5f264ea 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs @@ -85,61 +85,6 @@ fn test_parquet_pushdown() { ); } -#[test] -fn test_topk_pushdown() { - // filter should be pushed down into the parquet scan with two filters - let scan = parquet_scan(); - let predicate = col_lit_predicate("a", "foo", schema()); - let filter = - Arc::new(FilterExec::try_new(Arc::clone(&predicate), Arc::clone(&scan)).unwrap()); - let plan = Arc::new(SortExec::new( - LexOrdering::new(vec![PhysicalSortExpr::new( - Arc::new(Column::new_with_schema("a", schema()).unwrap()), - SortOptions::default(), - )]), - filter, - )); - - insta::assert_snapshot!( - OptimizationTest::new(plan, PushdownFilter{}), - @r" - OptimizationTest: - input: - - SortExec: expr=[a@0 ASC], preserve_partitioning=[false] - - FilterExec: a@0 = foo - - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=parquet - output: - Ok: - - SortExec: expr=[a@0 ASC], preserve_partitioning=[false] - - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=parquet, predicate=a@0 = foo AND DynamicFilterPhysicalExpr [ SortDynamicFilterSource[ ] ] - " - ); - - let sort = Arc::new(SortExec::new( - LexOrdering::new(vec![PhysicalSortExpr::new( - Arc::new(Column::new_with_schema("a", schema()).unwrap()), - SortOptions::default(), - )]), - Arc::clone(&scan), - )); - let plan = Arc::new(FilterExec::try_new(predicate, sort).unwrap()); - - insta::assert_snapshot!( - OptimizationTest::new(plan, PushdownFilter{}), - @r" - OptimizationTest: - input: - - FilterExec: a@0 = foo - - SortExec: expr=[a@0 ASC], preserve_partitioning=[false] - - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=parquet - output: - Ok: - - SortExec: expr=[a@0 ASC], preserve_partitioning=[false] - - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ SortDynamicFilterSource[ ] ] AND a@0 = foo - " - ); -} - /// Schema: /// a: String /// b: String From 7e2db6645065d1b70c165d2e6969da221b125177 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Thu, 3 Apr 2025 13:33:55 -0500 Subject: [PATCH 03/40] fix tests --- .../physical_optimizer/filter_pushdown.rs | 168 +++++++++++++++--- datafusion/datasource/src/file.rs | 12 +- datafusion/datasource/src/file_scan_config.rs | 24 ++- datafusion/datasource/src/source.rs | 51 +++++- datafusion/physical-expr/src/lib.rs | 2 +- datafusion/physical-expr/src/utils/mod.rs | 25 +++ .../physical-optimizer/src/filter_pushdown.rs | 6 +- .../physical-plan/src/execution_plan.rs | 6 +- datafusion/physical-plan/src/filter.rs | 97 +++++++++- .../sqllogictest/test_files/explain.slt | 3 + 10 files changed, 347 insertions(+), 47 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs index ef5bbf5f264ea..3877a3238a901 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs @@ -16,7 +16,6 @@ // under the License. use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; -use arrow_schema::SortOptions; use datafusion::{ datasource::object_store::ObjectStoreUrl, logical_expr::Operator, @@ -26,23 +25,149 @@ use datafusion::{ }, scalar::ScalarValue, }; -use datafusion_common::config::{ConfigOptions, TableParquetOptions}; use datafusion_common::internal_err; -use datafusion_datasource::file::FileSource; +use datafusion_common::{config::ConfigOptions, Statistics}; use datafusion_datasource::file_scan_config::FileScanConfigBuilder; use datafusion_datasource::source::DataSourceExec; -use datafusion_datasource_parquet::source::ParquetSource; -use datafusion_physical_expr::{LexOrdering, PhysicalSortExpr}; +use datafusion_datasource::{ + file::{FileSource, FileSourceFilterPushdownResult}, + file_scan_config::FileScanConfig, + file_stream::FileOpener, +}; +use datafusion_physical_expr::{conjunction, PhysicalExprRef}; +use datafusion_physical_expr_common::physical_expr::fmt_sql; use datafusion_physical_optimizer::filter_pushdown::PushdownFilter; use datafusion_physical_optimizer::PhysicalOptimizerRule; -use datafusion_physical_plan::{displayable, ExecutionPlan}; -use datafusion_physical_plan::{filter::FilterExec, sorts::sort::SortExec}; -use std::fmt::{Display, Formatter}; +use datafusion_physical_plan::filter::FilterExec; +use datafusion_physical_plan::{ + displayable, execution_plan::FilterSupport, metrics::ExecutionPlanMetricsSet, + DisplayFormatType, ExecutionPlan, +}; +use object_store::ObjectStore; use std::sync::{Arc, OnceLock}; +use std::{ + any::Any, + fmt::{Display, Formatter}, +}; + +/// A placeholder data source that accepts filter pushdown +#[derive(Clone)] +struct TestSource { + support: FilterSupport, + predicate: Option, + statistics: Option, +} + +impl TestSource { + fn new(support: FilterSupport) -> Self { + Self { + support, + predicate: None, + statistics: None, + } + } +} + +impl FileSource for TestSource { + fn create_file_opener( + &self, + _object_store: Arc, + _base_config: &FileScanConfig, + _partition: usize, + ) -> Arc { + todo!("should not be called") + } + + fn as_any(&self) -> &dyn Any { + todo!("should not be called") + } + + fn with_batch_size(&self, _batch_size: usize) -> Arc { + todo!("should not be called") + } + + fn with_schema(&self, _schema: SchemaRef) -> Arc { + todo!("should not be called") + } + + fn with_projection(&self, _config: &FileScanConfig) -> Arc { + todo!("should not be called") + } + + fn with_statistics(&self, statistics: Statistics) -> Arc { + Arc::new(TestSource { + statistics: Some(statistics), + ..self.clone() + }) + } + + fn metrics(&self) -> &ExecutionPlanMetricsSet { + todo!("should not be called") + } + + fn statistics(&self) -> datafusion_common::Result { + Ok(self + .statistics + .as_ref() + .expect("statistics not set") + .clone()) + } + + fn file_type(&self) -> &str { + "test" + } + + fn fmt_extra(&self, t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { + match t { + DisplayFormatType::Default | DisplayFormatType::Verbose => { + let predicate_string = self + .predicate + .as_ref() + .map(|p| format!(", predicate={p}")) + .unwrap_or_default(); + + write!(f, "{}", predicate_string) + } + DisplayFormatType::TreeRender => { + if let Some(predicate) = &self.predicate { + writeln!(f, "predicate={}", fmt_sql(predicate.as_ref()))?; + } + Ok(()) + } + } + } + + fn push_down_filters( + &self, + filters: &[PhysicalExprRef], + ) -> datafusion_common::Result> { + let new = Arc::new(TestSource { + support: self.support.clone(), + predicate: Some(conjunction(filters.iter().map(Arc::clone))), + statistics: self.statistics.clone(), + }); + Ok(Some(FileSourceFilterPushdownResult::new( + new, + vec![self.support; filters.len()], + ))) + } +} + +fn test_scan(support: FilterSupport) -> Arc { + let schema = schema(); + let source = Arc::new(TestSource::new(support)); + let base_config = FileScanConfigBuilder::new( + ObjectStoreUrl::parse("test://").unwrap(), + Arc::clone(schema), + source, + ) + .build(); + DataSourceExec::from_data_source(base_config) +} #[test] fn test_pushdown_into_scan() { - let scan = parquet_scan(); + let scan = test_scan(FilterSupport::HandledExact); let predicate = col_lit_predicate("a", "foo", schema()); let plan = Arc::new(FilterExec::try_new(predicate, scan).unwrap()); @@ -53,10 +178,10 @@ fn test_pushdown_into_scan() { OptimizationTest: input: - FilterExec: a@0 = foo - - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=parquet + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test output: Ok: - - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=parquet, predicate=a@0 = foo + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, predicate=a@0 = foo " ); } @@ -64,7 +189,7 @@ fn test_pushdown_into_scan() { #[test] fn test_parquet_pushdown() { // filter should be pushed down into the parquet scan with two filters - let scan = parquet_scan(); + let scan = test_scan(FilterSupport::HandledExact); let predicate1 = col_lit_predicate("a", "foo", schema()); let filter1 = Arc::new(FilterExec::try_new(predicate1, scan).unwrap()); let predicate2 = col_lit_predicate("b", "bar", schema()); @@ -77,10 +202,10 @@ fn test_parquet_pushdown() { input: - FilterExec: b@1 = bar - FilterExec: a@0 = foo - - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=parquet + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test output: Ok: - - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=parquet, predicate=a@0 = foo AND b@1 = bar + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, predicate=a@0 = foo AND b@1 = bar " ); } @@ -102,21 +227,6 @@ fn schema() -> &'static SchemaRef { }) } -/// Return a execution plan that reads from a parquet file -fn parquet_scan() -> Arc { - let schema = schema(); - let mut options = TableParquetOptions::default(); - options.global.pushdown_filters = true; - let source = ParquetSource::new(options).with_schema(Arc::clone(schema)); - let base_config = FileScanConfigBuilder::new( - ObjectStoreUrl::parse("test://").unwrap(), - Arc::clone(schema), - source, - ) - .build(); - DataSourceExec::from_data_source(base_config) -} - /// Returns a predicate that is a binary expression col = lit fn col_lit_predicate( column_name: &str, diff --git a/datafusion/datasource/src/file.rs b/datafusion/datasource/src/file.rs index 0066f39801a1b..bd09c1192dba1 100644 --- a/datafusion/datasource/src/file.rs +++ b/datafusion/datasource/src/file.rs @@ -27,7 +27,8 @@ use crate::file_scan_config::FileScanConfig; use crate::file_stream::FileOpener; use arrow::datatypes::SchemaRef; use datafusion_common::Statistics; -use datafusion_physical_expr::LexOrdering; +use datafusion_physical_expr::{LexOrdering, PhysicalExprRef}; +use datafusion_physical_plan::execution_plan::FilterPushdownResult; use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; use datafusion_physical_plan::DisplayFormatType; @@ -93,4 +94,13 @@ pub trait FileSource: Send + Sync { } Ok(None) } + + fn push_down_filters( + &self, + _filters: &[PhysicalExprRef], + ) -> datafusion_common::Result> { + Ok(None) + } } + +pub type FileSourceFilterPushdownResult = FilterPushdownResult>; diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index 729283289cafc..d71443867af8e 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -38,7 +38,7 @@ use datafusion_execution::{ }; use datafusion_physical_expr::{ expressions::Column, EquivalenceProperties, LexOrdering, Partitioning, - PhysicalSortExpr, + PhysicalExprRef, PhysicalSortExpr, }; use datafusion_physical_plan::{ display::{display_orderings, ProjectSchemaDisplay}, @@ -48,7 +48,6 @@ use datafusion_physical_plan::{ }; use log::{debug, warn}; -use crate::file_groups::FileGroup; use crate::{ display::FileGroupsDisplay, file::FileSource, @@ -58,6 +57,7 @@ use crate::{ statistics::MinMaxStatistics, PartitionedFile, }; +use crate::{file_groups::FileGroup, source::DataSourceFilterPushdownResult}; /// The base configurations for a [`DataSourceExec`], the a physical plan for /// any given file format. @@ -584,6 +584,22 @@ impl DataSource for FileScanConfig { ) as _ })) } + + fn push_down_filters( + &self, + filters: &[PhysicalExprRef], + ) -> Result> { + if let Some(file_source_result) = self.file_source.push_down_filters(filters)? { + let mut new_self = self.clone(); + new_self.file_source = file_source_result.inner; + Ok(Some(DataSourceFilterPushdownResult { + inner: Arc::new(new_self) as Arc, + support: file_source_result.support, + })) + } else { + Ok(None) + } + } } impl FileScanConfig { @@ -2000,7 +2016,7 @@ mod tests { }, partition_values: vec![ScalarValue::from(file.date)], range: None, - statistics: Some(Arc::new(Statistics { + statistics: Some(Statistics { num_rows: Precision::Absent, total_byte_size: Precision::Absent, column_statistics: file @@ -2020,7 +2036,7 @@ mod tests { .unwrap_or_default() }) .collect::>(), - })), + }), extensions: None, metadata_size_hint: None, } diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index 6c9122ce1ac10..159dbe1d5d181 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -22,7 +22,10 @@ use std::fmt; use std::fmt::{Debug, Formatter}; use std::sync::Arc; -use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; +use datafusion_physical_plan::execution_plan::{ + Boundedness, EmissionType, ExecutionPlanFilterPushdownResult, FilterPushdownResult, + FilterSupport, +}; use datafusion_physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use datafusion_physical_plan::projection::ProjectionExec; use datafusion_physical_plan::{ @@ -33,7 +36,7 @@ use crate::file_scan_config::FileScanConfig; use datafusion_common::config::ConfigOptions; use datafusion_common::{Constraints, Statistics}; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; -use datafusion_physical_expr::{EquivalenceProperties, Partitioning}; +use datafusion_physical_expr::{EquivalenceProperties, Partitioning, PhysicalExprRef}; use datafusion_physical_expr_common::sort_expr::LexOrdering; /// Common behaviors in Data Sources for both from Files and Memory. @@ -79,8 +82,20 @@ pub trait DataSource: Send + Sync + Debug { &self, _projection: &ProjectionExec, ) -> datafusion_common::Result>>; + /// Push down filters from parent execution plans to this data source. + /// This is expected to return Ok(None) if the filters cannot be pushed down. + /// If they can be pushed down it should return a [`FilterPushdownResult`] containing the new + /// data source and the support level for each filter (exact or inexact). + fn push_down_filters( + &self, + _filters: &[PhysicalExprRef], + ) -> datafusion_common::Result> { + Ok(None) + } } +pub type DataSourceFilterPushdownResult = FilterPushdownResult>; + /// [`ExecutionPlan`] handles different file formats like JSON, CSV, AVRO, ARROW, PARQUET /// /// `DataSourceExec` implements common functionality such as applying projections, @@ -192,6 +207,28 @@ impl ExecutionPlan for DataSourceExec { ) -> datafusion_common::Result>> { self.data_source.try_swapping_with_projection(projection) } + + fn with_filter_pushdown_result( + self: Arc, + own_filters_result: &[FilterSupport], + parent_filters_remaining: &[PhysicalExprRef], + ) -> datafusion_common::Result> { + // We didn't give out any filters, this should be empty! + assert!(own_filters_result.is_empty()); + // Forward filter pushdown to our data source. + if let Some(pushdown_result) = self + .data_source + .push_down_filters(parent_filters_remaining)? + { + let new_self = Arc::new(DataSourceExec::new(pushdown_result.inner)); + Ok(Some(ExecutionPlanFilterPushdownResult::new( + new_self, + pushdown_result.support, + ))) + } else { + Ok(None) + } + } } impl DataSourceExec { @@ -254,3 +291,13 @@ impl DataSourceExec { }) } } + +/// Create a new `DataSourceExec` from a `DataSource` +impl From for DataSourceExec +where + S: DataSource + 'static, +{ + fn from(source: S) -> Self { + Self::new(Arc::new(source)) + } +} diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index 93ced2eb628d8..9f795c81fa48e 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -68,7 +68,7 @@ pub use planner::{create_physical_expr, create_physical_exprs}; pub use scalar_function::ScalarFunctionExpr; pub use datafusion_physical_expr_common::utils::reverse_order_bys; -pub use utils::split_conjunction; +pub use utils::{conjunction, conjunction_opt, split_conjunction}; // For backwards compatibility pub mod tree_node { diff --git a/datafusion/physical-expr/src/utils/mod.rs b/datafusion/physical-expr/src/utils/mod.rs index 7e4c7f0e10ba8..b4d0758fd2e81 100644 --- a/datafusion/physical-expr/src/utils/mod.rs +++ b/datafusion/physical-expr/src/utils/mod.rs @@ -47,6 +47,31 @@ pub fn split_conjunction( split_impl(Operator::And, predicate, vec![]) } +/// Create a conjunction of the given predicates. +/// If the input is empty, return a literal true. +/// If the input contains a single predicate, return the predicate. +/// Otherwise, return a conjunction of the predicates (e.g. `a AND b AND c`). +pub fn conjunction( + predicates: impl IntoIterator>, +) -> Arc { + conjunction_opt(predicates).unwrap_or_else(|| crate::expressions::lit(true)) +} + +/// Create a conjunction of the given predicates. +/// If the input is empty or the return None. +/// If the input contains a single predicate, return Some(predicate). +/// Otherwise, return a Some(..) of a conjunction of the predicates (e.g. `Some(a AND b AND c)`). +pub fn conjunction_opt( + predicates: impl IntoIterator>, +) -> Option> { + predicates + .into_iter() + .fold(None, |acc, predicate| match acc { + None => Some(predicate), + Some(acc) => Some(Arc::new(BinaryExpr::new(acc, Operator::And, predicate))), + }) +} + /// Assume the predicate is in the form of DNF, split the predicate to a Vec of PhysicalExprs. /// /// For example, split "a1 = a2 OR b1 <= b2 OR c1 != c2" into ["a1 = a2", "b1 <= b2", "c1 != c2"] diff --git a/datafusion/physical-optimizer/src/filter_pushdown.rs b/datafusion/physical-optimizer/src/filter_pushdown.rs index 21610564306cd..1e13c6615c8b2 100644 --- a/datafusion/physical-optimizer/src/filter_pushdown.rs +++ b/datafusion/physical-optimizer/src/filter_pushdown.rs @@ -18,7 +18,7 @@ use std::sync::Arc; use datafusion_common::{config::ConfigOptions, Result}; -use datafusion_physical_expr::PhysicalExpr; +use datafusion_physical_expr::PhysicalExprRef; use datafusion_physical_plan::{ execution_plan::{ ExecutionPlanFilterPushdownResult, FilterPushdownAllowed, FilterSupport, @@ -73,7 +73,7 @@ impl ChildPushdownState { /// See [`pushdown_filters`] for more details. fn push_down_into_children( node: &Arc, - filters: &[Arc], + filters: &[PhysicalExprRef], ) -> Result { let children = node.children(); let mut new_children = Vec::with_capacity(children.len()); @@ -121,7 +121,7 @@ fn push_down_into_children( /// See [`PushdownFilter`] for more details on how this works in practice. fn pushdown_filters( node: &Arc, - parent_filters: &[Arc], + parent_filters: &[PhysicalExprRef], ) -> Result> { // Gather the filters from the current node. // These are the filters the current node "owns" or "produces" and wants to push down. diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index 595e1afd32c6a..ad154eb2c8e84 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -48,7 +48,7 @@ use datafusion_common::config::ConfigOptions; use datafusion_common::{exec_err, Constraints, Result}; use datafusion_common_runtime::JoinSet; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; +use datafusion_physical_expr::{EquivalenceProperties, LexOrdering, PhysicalExprRef}; use datafusion_physical_expr_common::sort_expr::LexRequirement; use futures::stream::{StreamExt, TryStreamExt}; @@ -496,7 +496,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// RepartitionExec nodes allow all filters to be pushed down as they don't change the schema or cardinality. fn filter_pushdown_request( &self, - filters: &[Arc], + filters: &[PhysicalExprRef], ) -> Result> { Ok(vec![FilterPushdownAllowed::Disallowed; filters.len()]) } @@ -507,7 +507,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { fn with_filter_pushdown_result( self: Arc, _own_filters_result: &[FilterSupport], - _parent_filters_remaining: &[Arc], + _parent_filters_remaining: &[PhysicalExprRef], ) -> Result> { Ok(None) } diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index a8a9973ea0434..6085680fc5a3f 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -25,7 +25,10 @@ use super::{ RecordBatchStream, SendableRecordBatchStream, Statistics, }; use crate::common::can_project; -use crate::execution_plan::CardinalityEffect; +use crate::execution_plan::{ + CardinalityEffect, ExecutionPlanFilterPushdownResult, FilterPushdownAllowed, + FilterSupport, +}; use crate::projection::{ make_with_child, try_embed_projection, update_expr, EmbeddedProjection, ProjectionExec, @@ -34,6 +37,7 @@ use crate::{ metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}, DisplayFormatType, ExecutionPlan, }; +use datafusion_physical_expr::expressions::lit; use arrow::compute::filter_record_batch; use arrow::datatypes::{DataType, SchemaRef}; @@ -48,10 +52,10 @@ use datafusion_expr::Operator; use datafusion_physical_expr::equivalence::ProjectionMapping; use datafusion_physical_expr::expressions::BinaryExpr; use datafusion_physical_expr::intervals::utils::check_support; -use datafusion_physical_expr::utils::collect_columns; +use datafusion_physical_expr::utils::{collect_columns, reassign_predicate_columns}; use datafusion_physical_expr::{ - analyze, split_conjunction, AcrossPartitions, AnalysisContext, ConstExpr, - ExprBoundaries, PhysicalExpr, + analyze, conjunction_opt, split_conjunction, AcrossPartitions, AnalysisContext, + ConstExpr, ExprBoundaries, PhysicalExpr, PhysicalExprRef, }; use datafusion_physical_expr_common::physical_expr::fmt_sql; @@ -433,6 +437,91 @@ impl ExecutionPlan for FilterExec { } try_embed_projection(projection, self) } + + fn filter_pushdown_request( + &self, + filters: &[PhysicalExprRef], + ) -> Result> { + let filters = if self.projection.is_some() { + let input_schema = self.input.schema(); + filters + .iter() + .map(|f| reassign_predicate_columns(Arc::clone(f), &input_schema, false)) + .collect::>>()? + } else { + filters.to_vec() + }; + + Ok(filters + .into_iter() + .map(FilterPushdownAllowed::Allowed) + .collect()) + } + + fn filters_for_pushdown(&self) -> Result>> { + let predicate = reassign_predicate_columns( + Arc::clone(&self.predicate), + &self.input.schema(), + false, + )?; + Ok(vec![predicate]) + } + + fn with_filter_pushdown_result( + self: Arc, + own_filters_result: &[FilterSupport], + parent_filters_remaining: &[PhysicalExprRef], + ) -> Result> { + // Only keep filters who's index maps to the pushdown result Unsupported + let filters_for_pushdown = self.filters_for_pushdown()?; + let new_filters = filters_for_pushdown + .iter() + .zip(own_filters_result.iter()) + .filter_map(|(f, p)| { + if matches!(p, FilterSupport::HandledExact) { + // Exact pushdown support means we keep discard filter + None + } else { + // Otherwise we still have to apply it + Some(Arc::clone(f)) + } + }) + // Combine that with any leftover filters from parents that our children couldn't handle + .chain(parent_filters_remaining.iter().map(Arc::clone)); + + let new_predicate = conjunction_opt(new_filters); + + if new_predicate.is_none() && self.projection.is_none() { + // We can remove ourselves from the execution tree + Ok(Some(ExecutionPlanFilterPushdownResult::new( + Arc::clone(&self.input), + vec![FilterSupport::HandledExact; parent_filters_remaining.len()], + ))) + } else { + // Even if we don't have a predicate we keep ourselves in the tree for our projection. + let new_predicate = new_predicate.unwrap_or_else(|| lit(true)); + let cache = Self::compute_properties( + &self.input, + &new_predicate, + self.default_selectivity, + self.projection.as_ref(), + )?; + Ok(Some(ExecutionPlanFilterPushdownResult { + inner: Arc::new(Self { + predicate: new_predicate, + input: Arc::clone(&self.input), + metrics: self.metrics.clone(), + default_selectivity: self.default_selectivity, + cache, + projection: self.projection.clone(), + }), + support: vec![ + FilterSupport::HandledExact; + parent_filters_remaining.len() + ], + })) + } + } } impl EmbeddedProjection for FilterExec { diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index deff793e51106..9f00ab8bc83a6 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -237,6 +237,7 @@ physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after coalesce_batches SAME TEXT AS ABOVE physical_plan after OutputRequirements DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true physical_plan after LimitAggregation SAME TEXT AS ABOVE +physical_plan after FilterPushdown SAME TEXT AS ABOVE physical_plan after LimitPushdown SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after SanityCheckPlan SAME TEXT AS ABOVE @@ -313,6 +314,7 @@ physical_plan after OutputRequirements 01)GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] 02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] physical_plan after LimitAggregation SAME TEXT AS ABOVE +physical_plan after FilterPushdown SAME TEXT AS ABOVE physical_plan after LimitPushdown DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after SanityCheckPlan SAME TEXT AS ABOVE @@ -353,6 +355,7 @@ physical_plan after OutputRequirements 01)GlobalLimitExec: skip=0, fetch=10 02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet physical_plan after LimitAggregation SAME TEXT AS ABOVE +physical_plan after FilterPushdown SAME TEXT AS ABOVE physical_plan after LimitPushdown DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after SanityCheckPlan SAME TEXT AS ABOVE From cb1f8300ba6759f84f3e145506f8c9fbb8a5257b Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Thu, 3 Apr 2025 13:37:24 -0500 Subject: [PATCH 04/40] fix --- datafusion/datasource/src/file_scan_config.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index d71443867af8e..9c3ed9e204171 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -2016,7 +2016,7 @@ mod tests { }, partition_values: vec![ScalarValue::from(file.date)], range: None, - statistics: Some(Statistics { + statistics: Some(Arc::new(Statistics { num_rows: Precision::Absent, total_byte_size: Precision::Absent, column_statistics: file @@ -2036,7 +2036,7 @@ mod tests { .unwrap_or_default() }) .collect::>(), - }), + })), extensions: None, metadata_size_hint: None, } From e92d8b5f5102f890659ae35d0668eeca8b3b5f4f Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Thu, 3 Apr 2025 13:55:06 -0500 Subject: [PATCH 05/40] fix --- .../core/tests/physical_optimizer/filter_pushdown.rs | 2 +- datafusion/physical-optimizer/src/filter_pushdown.rs | 8 +++++--- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs index 3877a3238a901..a02e6cbd5501c 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs @@ -142,7 +142,7 @@ impl FileSource for TestSource { filters: &[PhysicalExprRef], ) -> datafusion_common::Result> { let new = Arc::new(TestSource { - support: self.support.clone(), + support: self.support, predicate: Some(conjunction(filters.iter().map(Arc::clone))), statistics: self.statistics.clone(), }); diff --git a/datafusion/physical-optimizer/src/filter_pushdown.rs b/datafusion/physical-optimizer/src/filter_pushdown.rs index 1e13c6615c8b2..9a0466abc66cf 100644 --- a/datafusion/physical-optimizer/src/filter_pushdown.rs +++ b/datafusion/physical-optimizer/src/filter_pushdown.rs @@ -418,9 +418,11 @@ fn pushdown_filters( /// You may notice in this case that the filter is *dynamic*: the hash table is built /// _after_ the `departments` table is read and at runtime. /// We don't have a concrete `InList` filter or similar to push down at optimization time. -/// These sorts of dynamic filters are handled by building a specialized [`PhysicalExpr`] that -/// internally maintains a reference to the hash table or other state. -/// To make working with these sorts of dynamic filters more tractable we have the method [`PhysicalExpr::snapshot`] +/// These sorts of dynamic filters are handled by building a specialized +/// [`PhysicalExpr`][datafusion_physical_expr::PhysicalExpr] that can be evaluated at runtime +/// and internally maintains a reference to the hash table or other state. +/// To make working with these sorts of dynamic filters more tractable we have the method `PhysicalExpr::snapshot` +/// (TODO: add reference after https://github.com/apache/datafusion/pull/15568 is merged) /// which attempts to simplify a dynamic filter into a "basic" non-dynamic filter. /// For a join this could mean converting it to an `InList` filter or a min/max filter for example. /// See `datafusion/physical-plan/src/dynamic_filters.rs` for more details. From ca391c1aef4254535000b384577b64f87fd08586 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Thu, 3 Apr 2025 14:19:16 -0500 Subject: [PATCH 06/40] fix doc --- datafusion/physical-optimizer/src/filter_pushdown.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/physical-optimizer/src/filter_pushdown.rs b/datafusion/physical-optimizer/src/filter_pushdown.rs index 9a0466abc66cf..72b16c024cf9b 100644 --- a/datafusion/physical-optimizer/src/filter_pushdown.rs +++ b/datafusion/physical-optimizer/src/filter_pushdown.rs @@ -422,7 +422,7 @@ fn pushdown_filters( /// [`PhysicalExpr`][datafusion_physical_expr::PhysicalExpr] that can be evaluated at runtime /// and internally maintains a reference to the hash table or other state. /// To make working with these sorts of dynamic filters more tractable we have the method `PhysicalExpr::snapshot` -/// (TODO: add reference after https://github.com/apache/datafusion/pull/15568 is merged) +/// (TODO: add reference after is merged) /// which attempts to simplify a dynamic filter into a "basic" non-dynamic filter. /// For a join this could mean converting it to an `InList` filter or a min/max filter for example. /// See `datafusion/physical-plan/src/dynamic_filters.rs` for more details. From c78a5902d52507d2e530fcf82d365f49edb7770f Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Thu, 3 Apr 2025 15:02:45 -0500 Subject: [PATCH 07/40] fix doc --- datafusion/physical-plan/src/execution_plan.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index ad154eb2c8e84..7c519bd404ad2 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -483,7 +483,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// That is, a query such as `WHERE a = 1 AND b = 2` would return two /// filters: `a = 1` and `b = 2`. /// They can always be assembled into a single filter using - /// [`crate::physical_expr::split_conjunction`]. + /// [`split_conjunction`][datafusion_physical_expr::split_conjunction]. fn filters_for_pushdown(&self) -> Result>> { Ok(Vec::new()) } From 34c8285bee94de7fd06c0d85ce8d1673f391d765 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sat, 5 Apr 2025 10:24:42 -0400 Subject: [PATCH 08/40] Improve doc comments of `filter-pushdown-apis` (#22) * Improve doc comments * Apply suggestions from code review --------- Co-authored-by: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> --- datafusion/datasource/src/file.rs | 6 +++ datafusion/datasource/src/source.rs | 42 ++++++++++++------- .../physical-plan/src/execution_plan.rs | 25 ++++++----- 3 files changed, 47 insertions(+), 26 deletions(-) diff --git a/datafusion/datasource/src/file.rs b/datafusion/datasource/src/file.rs index bd09c1192dba1..ceea81feaf7b1 100644 --- a/datafusion/datasource/src/file.rs +++ b/datafusion/datasource/src/file.rs @@ -95,6 +95,12 @@ pub trait FileSource: Send + Sync { Ok(None) } + /// Push down filters to the file source if supported. + /// + /// Returns `Ok(None)` by default. See [`ExecutionPlan::with_filter_pushdown_result`] + /// for more details. + /// + /// [`ExecutionPlan::with_filter_pushdown_result`]: datafusion_physical_plan::execution_plan::ExecutionPlan::with_filter_pushdown_result fn push_down_filters( &self, _filters: &[PhysicalExprRef], diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index 159dbe1d5d181..6dc3d47e31f47 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -34,7 +34,7 @@ use datafusion_physical_plan::{ use crate::file_scan_config::FileScanConfig; use datafusion_common::config::ConfigOptions; -use datafusion_common::{Constraints, Statistics}; +use datafusion_common::{Constraints, Result, Statistics}; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_physical_expr::{EquivalenceProperties, Partitioning, PhysicalExprRef}; use datafusion_physical_expr_common::sort_expr::LexOrdering; @@ -54,7 +54,7 @@ pub trait DataSource: Send + Sync + Debug { &self, partition: usize, context: Arc, - ) -> datafusion_common::Result; + ) -> Result; fn as_any(&self) -> &dyn Any; /// Format this source for display in explain plans fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result; @@ -65,13 +65,13 @@ pub trait DataSource: Send + Sync + Debug { _target_partitions: usize, _repartition_file_min_size: usize, _output_ordering: Option, - ) -> datafusion_common::Result>> { + ) -> Result>> { Ok(None) } fn output_partitioning(&self) -> Partitioning; fn eq_properties(&self) -> EquivalenceProperties; - fn statistics(&self) -> datafusion_common::Result; + fn statistics(&self) -> Result; /// Return a copy of this DataSource with a new fetch limit fn with_fetch(&self, _limit: Option) -> Option>; fn fetch(&self) -> Option; @@ -81,15 +81,25 @@ pub trait DataSource: Send + Sync + Debug { fn try_swapping_with_projection( &self, _projection: &ProjectionExec, - ) -> datafusion_common::Result>>; - /// Push down filters from parent execution plans to this data source. - /// This is expected to return Ok(None) if the filters cannot be pushed down. - /// If they can be pushed down it should return a [`FilterPushdownResult`] containing the new - /// data source and the support level for each filter (exact or inexact). + ) -> Result>>; + + /// Push down filters into this `DataSource`. + /// + /// Returns `Ok(None)` if the filters cannot be evaluated within the + /// `DataSource`. + /// + /// If the filters can be evaluated by the `DataSource`, + /// return a [`FilterPushdownResult`] containing an updated + /// `DataSource` and the support level for each filter (exact or inexact). + /// + /// Default implementation returns `Ok(None)`. See [`ExecutionPlan::with_filter_pushdown_result`] + /// for more details. + /// + /// [`ExecutionPlan::push_down_filters`]: datafusion_physical_plan::execution_plan::ExecutionPlan::with_filter_pushdown_result fn push_down_filters( &self, _filters: &[PhysicalExprRef], - ) -> datafusion_common::Result> { + ) -> Result>> { Ok(None) } } @@ -146,7 +156,7 @@ impl ExecutionPlan for DataSourceExec { fn with_new_children( self: Arc, _: Vec>, - ) -> datafusion_common::Result> { + ) -> Result> { Ok(self) } @@ -154,7 +164,7 @@ impl ExecutionPlan for DataSourceExec { &self, target_partitions: usize, config: &ConfigOptions, - ) -> datafusion_common::Result>> { + ) -> Result>> { let data_source = self.data_source.repartitioned( target_partitions, config.optimizer.repartition_file_min_size, @@ -178,7 +188,7 @@ impl ExecutionPlan for DataSourceExec { &self, partition: usize, context: Arc, - ) -> datafusion_common::Result { + ) -> Result { self.data_source.open(partition, context) } @@ -186,7 +196,7 @@ impl ExecutionPlan for DataSourceExec { Some(self.data_source.metrics().clone_inner()) } - fn statistics(&self) -> datafusion_common::Result { + fn statistics(&self) -> Result { self.data_source.statistics() } @@ -204,7 +214,7 @@ impl ExecutionPlan for DataSourceExec { fn try_swapping_with_projection( &self, projection: &ProjectionExec, - ) -> datafusion_common::Result>> { + ) -> Result>> { self.data_source.try_swapping_with_projection(projection) } @@ -212,7 +222,7 @@ impl ExecutionPlan for DataSourceExec { self: Arc, own_filters_result: &[FilterSupport], parent_filters_remaining: &[PhysicalExprRef], - ) -> datafusion_common::Result> { + ) -> Result> { // We didn't give out any filters, this should be empty! assert!(own_filters_result.is_empty()); // Forward filter pushdown to our data source. diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index 7c519bd404ad2..1ea6c14f4d2a7 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -469,27 +469,32 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { } /// Returns a set of filters that this operator owns but would like to be pushed down. - /// For example, a `TopK` operator may produce dynamic filters that reference it's currrent state, - /// while a `FilterExec` will just hand of the filters it has as is. - /// The default implementation returns an empty vector. - /// These filters are applied row-by row and any that return `false` or `NULL` will be - /// filtered out and any that return `true` will be kept. - /// The expressions returned **must** always return `true` or `false`; - /// other truthy or falsy values are not allowed (e.g. `0`, `1`). + /// + /// For example, a `TopK` operator may produce dynamic filters that + /// reference it's current state, while a `FilterExec` will just hand of the + /// filters it has as is. + /// + /// The default implementation returns an empty vector. These filters are + /// applied row-by row: + /// 1. any that return `false` or `NULL` will be filtered out + /// 2. any that return `true` will be kept. + /// + /// The expressions returned **must** always be Boolean ( `true`, `false` or + /// NULL); other truthy or falsy values are not allowed (e.g. `0`, `1`). /// /// # Returns /// A vector of filters that this operator would like to push down. /// These should be treated as the split conjunction of a `WHERE` clause. /// That is, a query such as `WHERE a = 1 AND b = 2` would return two /// filters: `a = 1` and `b = 2`. - /// They can always be assembled into a single filter using - /// [`split_conjunction`][datafusion_physical_expr::split_conjunction]. + /// They can be combined into a single filter using + /// [`conjunction`][datafusion_physical_expr::conjunction]. fn filters_for_pushdown(&self) -> Result>> { Ok(Vec::new()) } /// Checks which filters this node allows to be pushed down through it from a parent to a child. - /// For example, a `ProjectionExec` node can allow filters that only refernece + /// For example, a `ProjectionExec` node can allow filters that only reference /// columns it did not create through but filters that reference columns it is creating cannot be pushed down any further. /// That is, it only allows some filters through because it changes the schema of the data. /// Aggregation nodes may not allow any filters to be pushed down as they change the cardinality of the data. From e15374f9d4e47f305259447f50ac725fa5a35358 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Sat, 5 Apr 2025 09:35:35 -0500 Subject: [PATCH 09/40] Apply suggestions from code review Co-authored-by: Andrew Lamb --- datafusion/core/tests/physical_optimizer/filter_pushdown.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs index a02e6cbd5501c..a153ed4dfef2e 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs @@ -187,7 +187,7 @@ fn test_pushdown_into_scan() { } #[test] -fn test_parquet_pushdown() { +test_filter_collapse() { // filter should be pushed down into the parquet scan with two filters let scan = test_scan(FilterSupport::HandledExact); let predicate1 = col_lit_predicate("a", "foo", schema()); From 2ceec35c3c09213f5fddc348f41301e201c298df Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Sat, 5 Apr 2025 15:24:22 -0500 Subject: [PATCH 10/40] simplify according to pr feedback --- .../physical_optimizer/filter_pushdown.rs | 30 +- datafusion/datasource/src/file.rs | 20 +- datafusion/datasource/src/file_scan_config.rs | 30 +- datafusion/datasource/src/source.rs | 62 +-- .../physical-optimizer/src/filter_pushdown.rs | 463 +----------------- .../physical-plan/src/execution_plan.rs | 414 ++++++++++++---- datafusion/physical-plan/src/filter.rs | 148 +++--- datafusion/physical-plan/src/lib.rs | 5 +- .../sqllogictest/test_files/explain.slt | 6 +- 9 files changed, 467 insertions(+), 711 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs index a153ed4dfef2e..b8dfa4d209fac 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs @@ -25,8 +25,8 @@ use datafusion::{ }, scalar::ScalarValue, }; -use datafusion_common::internal_err; use datafusion_common::{config::ConfigOptions, Statistics}; +use datafusion_common::{internal_err, Result}; use datafusion_datasource::file_scan_config::FileScanConfigBuilder; use datafusion_datasource::source::DataSourceExec; use datafusion_datasource::{ @@ -40,8 +40,8 @@ use datafusion_physical_optimizer::filter_pushdown::PushdownFilter; use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::filter::FilterExec; use datafusion_physical_plan::{ - displayable, execution_plan::FilterSupport, metrics::ExecutionPlanMetricsSet, - DisplayFormatType, ExecutionPlan, + displayable, filter_pushdown::FilterPushdownSupport, + metrics::ExecutionPlanMetricsSet, DisplayFormatType, ExecutionPlan, }; use object_store::ObjectStore; use std::sync::{Arc, OnceLock}; @@ -53,13 +53,13 @@ use std::{ /// A placeholder data source that accepts filter pushdown #[derive(Clone)] struct TestSource { - support: FilterSupport, + support: FilterPushdownSupport, predicate: Option, statistics: Option, } impl TestSource { - fn new(support: FilterSupport) -> Self { + fn new(support: FilterPushdownSupport) -> Self { Self { support, predicate: None, @@ -105,7 +105,7 @@ impl FileSource for TestSource { todo!("should not be called") } - fn statistics(&self) -> datafusion_common::Result { + fn statistics(&self) -> Result { Ok(self .statistics .as_ref() @@ -137,23 +137,23 @@ impl FileSource for TestSource { } } - fn push_down_filters( + fn try_pushdown_filters( &self, filters: &[PhysicalExprRef], - ) -> datafusion_common::Result> { + ) -> Result { let new = Arc::new(TestSource { support: self.support, predicate: Some(conjunction(filters.iter().map(Arc::clone))), statistics: self.statistics.clone(), }); - Ok(Some(FileSourceFilterPushdownResult::new( + Ok(FileSourceFilterPushdownResult::new( new, vec![self.support; filters.len()], - ))) + )) } } -fn test_scan(support: FilterSupport) -> Arc { +fn test_scan(support: FilterPushdownSupport) -> Arc { let schema = schema(); let source = Arc::new(TestSource::new(support)); let base_config = FileScanConfigBuilder::new( @@ -167,7 +167,7 @@ fn test_scan(support: FilterSupport) -> Arc { #[test] fn test_pushdown_into_scan() { - let scan = test_scan(FilterSupport::HandledExact); + let scan = test_scan(FilterPushdownSupport::Exact); let predicate = col_lit_predicate("a", "foo", schema()); let plan = Arc::new(FilterExec::try_new(predicate, scan).unwrap()); @@ -187,9 +187,9 @@ fn test_pushdown_into_scan() { } #[test] -test_filter_collapse() { +fn test_filter_collapse() { // filter should be pushed down into the parquet scan with two filters - let scan = test_scan(FilterSupport::HandledExact); + let scan = test_scan(FilterPushdownSupport::Exact); let predicate1 = col_lit_predicate("a", "foo", schema()); let filter1 = Arc::new(FilterExec::try_new(predicate1, scan).unwrap()); let predicate2 = col_lit_predicate("b", "bar", schema()); @@ -205,7 +205,7 @@ test_filter_collapse() { - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test output: Ok: - - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, predicate=a@0 = foo AND b@1 = bar + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, predicate=b@1 = bar AND a@0 = foo " ); } diff --git a/datafusion/datasource/src/file.rs b/datafusion/datasource/src/file.rs index ceea81feaf7b1..98420c4c88c68 100644 --- a/datafusion/datasource/src/file.rs +++ b/datafusion/datasource/src/file.rs @@ -26,9 +26,9 @@ use crate::file_groups::FileGroupPartitioner; use crate::file_scan_config::FileScanConfig; use crate::file_stream::FileOpener; use arrow::datatypes::SchemaRef; -use datafusion_common::Statistics; +use datafusion_common::{Result, Statistics}; use datafusion_physical_expr::{LexOrdering, PhysicalExprRef}; -use datafusion_physical_plan::execution_plan::FilterPushdownResult; +use datafusion_physical_plan::filter_pushdown::FilterPushdownResult; use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; use datafusion_physical_plan::DisplayFormatType; @@ -58,7 +58,7 @@ pub trait FileSource: Send + Sync { /// Return execution plan metrics fn metrics(&self) -> &ExecutionPlanMetricsSet; /// Return projected statistics - fn statistics(&self) -> datafusion_common::Result; + fn statistics(&self) -> Result; /// String representation of file source such as "csv", "json", "parquet" fn file_type(&self) -> &str; /// Format FileType specific information @@ -76,7 +76,7 @@ pub trait FileSource: Send + Sync { repartition_file_min_size: usize, output_ordering: Option, config: &FileScanConfig, - ) -> datafusion_common::Result> { + ) -> Result> { if config.file_compression_type.is_compressed() || config.new_lines_in_values { return Ok(None); } @@ -95,17 +95,11 @@ pub trait FileSource: Send + Sync { Ok(None) } - /// Push down filters to the file source if supported. - /// - /// Returns `Ok(None)` by default. See [`ExecutionPlan::with_filter_pushdown_result`] - /// for more details. - /// - /// [`ExecutionPlan::with_filter_pushdown_result`]: datafusion_physical_plan::execution_plan::ExecutionPlan::with_filter_pushdown_result - fn push_down_filters( + fn try_pushdown_filters( &self, _filters: &[PhysicalExprRef], - ) -> datafusion_common::Result> { - Ok(None) + ) -> Result { + Ok(FileSourceFilterPushdownResult::NotPushed) } } diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index 9c3ed9e204171..82e23c7370adb 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -50,7 +50,7 @@ use log::{debug, warn}; use crate::{ display::FileGroupsDisplay, - file::FileSource, + file::{FileSource, FileSourceFilterPushdownResult}, file_compression_type::FileCompressionType, file_stream::FileStream, source::{DataSource, DataSourceExec}, @@ -585,19 +585,25 @@ impl DataSource for FileScanConfig { })) } - fn push_down_filters( + fn try_pushdown_filters( &self, filters: &[PhysicalExprRef], - ) -> Result> { - if let Some(file_source_result) = self.file_source.push_down_filters(filters)? { - let mut new_self = self.clone(); - new_self.file_source = file_source_result.inner; - Ok(Some(DataSourceFilterPushdownResult { - inner: Arc::new(new_self) as Arc, - support: file_source_result.support, - })) - } else { - Ok(None) + ) -> Result { + match self.file_source.try_pushdown_filters(filters)? { + FileSourceFilterPushdownResult::NotPushed => { + Ok(DataSourceFilterPushdownResult::NotPushed) + } + FileSourceFilterPushdownResult::Pushed { inner, support } => { + let new_self = Arc::new( + FileScanConfigBuilder::from(self.clone()) + .with_source(inner) + .build(), + ); + Ok(DataSourceFilterPushdownResult::Pushed { + inner: new_self, + support, + }) + } } } } diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index 6dc3d47e31f47..453762b7a656a 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -22,14 +22,12 @@ use std::fmt; use std::fmt::{Debug, Formatter}; use std::sync::Arc; -use datafusion_physical_plan::execution_plan::{ - Boundedness, EmissionType, ExecutionPlanFilterPushdownResult, FilterPushdownResult, - FilterSupport, -}; +use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; use datafusion_physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use datafusion_physical_plan::projection::ProjectionExec; use datafusion_physical_plan::{ - DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, + DisplayAs, DisplayFormatType, ExecutionPlan, ExecutionPlanFilterPushdownResult, + FilterPushdownResult, PlanProperties, }; use crate::file_scan_config::FileScanConfig; @@ -83,24 +81,11 @@ pub trait DataSource: Send + Sync + Debug { _projection: &ProjectionExec, ) -> Result>>; - /// Push down filters into this `DataSource`. - /// - /// Returns `Ok(None)` if the filters cannot be evaluated within the - /// `DataSource`. - /// - /// If the filters can be evaluated by the `DataSource`, - /// return a [`FilterPushdownResult`] containing an updated - /// `DataSource` and the support level for each filter (exact or inexact). - /// - /// Default implementation returns `Ok(None)`. See [`ExecutionPlan::with_filter_pushdown_result`] - /// for more details. - /// - /// [`ExecutionPlan::push_down_filters`]: datafusion_physical_plan::execution_plan::ExecutionPlan::with_filter_pushdown_result - fn push_down_filters( + fn try_pushdown_filters( &self, _filters: &[PhysicalExprRef], - ) -> Result>> { - Ok(None) + ) -> Result { + Ok(DataSourceFilterPushdownResult::NotPushed) } } @@ -218,25 +203,22 @@ impl ExecutionPlan for DataSourceExec { self.data_source.try_swapping_with_projection(projection) } - fn with_filter_pushdown_result( - self: Arc, - own_filters_result: &[FilterSupport], - parent_filters_remaining: &[PhysicalExprRef], - ) -> Result> { - // We didn't give out any filters, this should be empty! - assert!(own_filters_result.is_empty()); - // Forward filter pushdown to our data source. - if let Some(pushdown_result) = self - .data_source - .push_down_filters(parent_filters_remaining)? - { - let new_self = Arc::new(DataSourceExec::new(pushdown_result.inner)); - Ok(Some(ExecutionPlanFilterPushdownResult::new( - new_self, - pushdown_result.support, - ))) - } else { - Ok(None) + fn try_pushdown_filters( + &self, + _plan: &Arc, + parent_filters: &[PhysicalExprRef], + ) -> Result { + match self.data_source.try_pushdown_filters(parent_filters)? { + DataSourceFilterPushdownResult::NotPushed => { + Ok(ExecutionPlanFilterPushdownResult::NotPushed) + } + DataSourceFilterPushdownResult::Pushed { inner, support } => { + let new_self = Arc::new(DataSourceExec::new(inner)); + Ok(ExecutionPlanFilterPushdownResult::Pushed { + inner: new_self, + support, + }) + } } } } diff --git a/datafusion/physical-optimizer/src/filter_pushdown.rs b/datafusion/physical-optimizer/src/filter_pushdown.rs index 72b16c024cf9b..4e2c9f3a9507f 100644 --- a/datafusion/physical-optimizer/src/filter_pushdown.rs +++ b/datafusion/physical-optimizer/src/filter_pushdown.rs @@ -17,454 +17,15 @@ use std::sync::Arc; -use datafusion_common::{config::ConfigOptions, Result}; -use datafusion_physical_expr::PhysicalExprRef; +use datafusion_common::{config::ConfigOptions, DataFusionError, Result}; use datafusion_physical_plan::{ - execution_plan::{ - ExecutionPlanFilterPushdownResult, FilterPushdownAllowed, FilterSupport, - }, - with_new_children_if_necessary, ExecutionPlan, + execution_plan::ExecutionPlanFilterPushdownResult, ExecutionPlan, }; use crate::PhysicalOptimizerRule; -/// The state of filter pushdown support for a given filter. -#[derive(Clone, Copy, Debug)] -enum ChildPushdownState { - /// A child said it can handle the filter exactly. - ChildExact, - /// A child exists and took a look at the filter. - /// It may partially handle it or not handle it at all. - /// The parent still needs to re-apply the filter. - ChildInexact, - /// No child exists, there is no one to handle the filter. - /// This is the default / initial state. - NoChild, -} - -impl ChildPushdownState { - /// Combine the current state with another state. - /// This is used to combine the results of multiple children. - fn combine_with_other(&self, other: &FilterSupport) -> ChildPushdownState { - match (other, self) { - (FilterSupport::HandledExact, ChildPushdownState::NoChild) => { - ChildPushdownState::ChildExact - } - (FilterSupport::HandledExact, ChildPushdownState::ChildInexact) => { - ChildPushdownState::ChildInexact - } - (FilterSupport::Unhandled, ChildPushdownState::NoChild) => { - ChildPushdownState::ChildInexact - } - (FilterSupport::Unhandled, ChildPushdownState::ChildExact) => { - ChildPushdownState::ChildInexact - } - (FilterSupport::Unhandled, ChildPushdownState::ChildInexact) => { - ChildPushdownState::ChildInexact - } - (FilterSupport::HandledExact, ChildPushdownState::ChildExact) => { - // If both are exact, keep it as exact - ChildPushdownState::ChildExact - } - } - } -} - -/// See [`pushdown_filters`] for more details. -fn push_down_into_children( - node: &Arc, - filters: &[PhysicalExprRef], -) -> Result { - let children = node.children(); - let mut new_children = Vec::with_capacity(children.len()); - let mut filter_pushdown_result = vec![ChildPushdownState::NoChild; filters.len()]; - for child in children { - if let Some(result) = pushdown_filters(child, filters)? { - new_children.push(result.inner); - for (idx, support) in result.support.iter().enumerate() { - filter_pushdown_result[idx] = - filter_pushdown_result[idx].combine_with_other(support) - } - } else { - new_children.push(Arc::clone(child)); - } - } - let support = filter_pushdown_result - .iter() - .map(|s| match s { - ChildPushdownState::ChildExact => FilterSupport::HandledExact, - ChildPushdownState::ChildInexact => FilterSupport::Unhandled, - ChildPushdownState::NoChild => FilterSupport::Unhandled, - }) - .collect::>(); - let node = with_new_children_if_necessary(Arc::clone(node), new_children)?; - Ok(ExecutionPlanFilterPushdownResult::new(node, support)) -} - -/// Recursively a collection of filters down through the execution plan tree in a depth-first manner. -/// -/// For each filter we try to push it down to children as far down as possible, keeping track of if the children -/// can handle the filter or not. -/// -/// If a child can handle the filter, we mark it as handled exact and parent nodes (including the source of the filter) -/// can decide to discard it / not re-apply it themselves. -/// If a child cannot handle the filter or may return false positives (aka "inexact" handling) we mark it as handled inexact. -/// If a child does not allow filter pushdown at all (e.g. an aggregation node) we keep recursing but clear the current set of filters -/// we are pushing down. -/// -/// As we recurse back up the tree we combine the results of the children to determine if the overall result is exact or inexact: -/// - For nodes with a single child we just take the child's result. -/// - For nodes with multiple children we combine the results of the children to determine if the overall result is exact or inexact. -/// We do this by checking if all children are exact (we return exact up) or if any child is inexact (we return inexact). -/// - If a node has no children this is equivalent to inexact handling (there is no child to handle the filter). -/// -/// See [`PushdownFilter`] for more details on how this works in practice. -fn pushdown_filters( - node: &Arc, - parent_filters: &[PhysicalExprRef], -) -> Result> { - // Gather the filters from the current node. - // These are the filters the current node "owns" or "produces" and wants to push down. - let node_filters = node.filters_for_pushdown()?; - // Check which nodes from parents this node is okay with us trying to push down to it's children. - let parent_pushdown_request_result = node.filter_pushdown_request(parent_filters)?; - // Do some index masking so that we only ever call nodes with the filters relevant to them / that they're allowed to touch. - // But we still need to reconstruct the full result for our caller. - let parent_filter_for_pushdown_indices = parent_pushdown_request_result - .iter() - .enumerate() - .filter_map(|(i, s)| { - if matches!(s, FilterPushdownAllowed::Allowed(_)) { - Some(i) - } else { - None - } - }) - .collect::>(); - let parent_filters_to_push_down = parent_filter_for_pushdown_indices - .iter() - .map(|&i| Arc::clone(&parent_filters[i])) - .collect::>(); - let all_filters_to_push_down = node_filters - .iter() - .chain(parent_filters_to_push_down.iter()) - .map(Arc::clone) - .collect::>(); - // Push down into children - let child_pushdown_result = push_down_into_children(node, &all_filters_to_push_down)?; - let mut node = child_pushdown_result.inner; - // A bit more index masking to construct the final result for our caller. - let node_filters_pushdown_result = - child_pushdown_result.support[..node_filters.len()].to_vec(); - let mut parent_filter_pushdown_result = - vec![FilterSupport::Unhandled; parent_filters.len()]; - for (parent_filter_idx, support) in parent_filter_for_pushdown_indices - .iter() - .zip(child_pushdown_result.support[node_filters.len()..].iter()) - { - parent_filter_pushdown_result[*parent_filter_idx] = *support; - } - // Collect the remaining unhandled parent filters - let unhandled_parent_filter_indices = (0..parent_filters.len()) - .filter(|&i| matches!(parent_filter_pushdown_result[i], FilterSupport::Unhandled)) - .collect::>(); - let unhandled_parent_filters = unhandled_parent_filter_indices - .iter() - .map(|&i| Arc::clone(&parent_filters[i])) - .collect::>(); - // Check if the node can handle the filters - if let Some(result) = Arc::clone(&node).with_filter_pushdown_result( - &node_filters_pushdown_result, - &unhandled_parent_filters, - )? { - node = result.inner; - for (parent_filter_index, support) in - unhandled_parent_filter_indices.iter().zip(result.support) - { - parent_filter_pushdown_result[*parent_filter_index] = support; - } - } - Ok(Some(ExecutionPlanFilterPushdownResult::new( - node, - parent_filter_pushdown_result, - ))) -} - /// A physical optimizer rule that pushes down filters in the execution plan. -/// For example, consider the following plan: -/// -/// ```text -/// ┌──────────────────────┐ -/// │ CoalesceBatchesExec │ -/// └──────────────────────┘ -/// │ -/// ▼ -/// ┌──────────────────────┐ -/// │ FilterExec │ -/// │ filters = [ id=1] │ -/// └──────────────────────┘ -/// │ -/// ▼ -/// ┌──────────────────────┐ -/// │ DataSourceExec │ -/// │ projection = * │ -/// └──────────────────────┘ -/// ``` -/// -/// Our goal is to move the `id = 1` filter from the `FilterExec` node to the `DataSourceExec` node. -/// If this filter is selective it can avoid massive amounts of data being read from the source (the projection is `*` so all matching columns are read). -/// In this simple case we: -/// 1. Enter the recursion with no filters. -/// 2. We find the `FilterExec` node and it tells us that it has a filter (see [`ExecutionPlan::filters_for_pushdown`] and `datafusion::physical_plan::filter::FilterExec`). -/// 3. We recurse down into it's children (the `DataSourceExec` node) now carrying the filters `[id = 1]`. -/// 4. The `DataSourceExec` node tells us that it can handle the filter and we mark it as handled exact (see [`ExecutionPlan::with_filter_pushdown_result`]). -/// 5. Since the `DataSourceExec` node has no children we recurse back up the tree. -/// 6. We now tell the `FilterExec` node that it has a child that can handle the filter and we mark it as handled exact (see [`ExecutionPlan::with_filter_pushdown_result`]). -/// The `FilterExec` node can now return a new execution plan, either a copy of itself without that filter or if has no work left to do it can even return the child node directly. -/// 7. We recurse back up to `CoalesceBatchesExec` and do nothing there since it had no filters to push down. -/// -/// The new plan looks like: -/// -/// ```text -/// ┌──────────────────────┐ -/// │ CoalesceBatchesExec │ -/// └──────────────────────┘ -/// │ -/// ▼ -/// ┌──────────────────────┐ -/// │ DataSourceExec │ -// │ projection = * │ -// │ filters = [ id=1] │ -/// └──────────────────────┘ -/// ``` -/// -/// Let's consider a more complex example involving a `ProjectionExec` node in betweeen the `FilterExec` and `DataSourceExec` nodes that creates a new column that the filter depends on. -/// -/// ```text -// ┌──────────────────────┐ -// │ CoalesceBatchesExec │ -// └──────────────────────┘ -// │ -// ▼ -// ┌──────────────────────┐ -// │ FilterExec │ -// │ filters = │ -// │ [cost>50,id=1] │ -// └──────────────────────┘ -// │ -// ▼ -// ┌──────────────────────┐ -// │ ProjectionExec │ -// │ cost = price * 1.2 │ -// └──────────────────────┘ -// │ -// ▼ -// ┌──────────────────────┐ -// │ DataSourceExec │ -// │ projection = * │ -// └──────────────────────┘ -/// ``` -/// -/// We want to push down the filters [id=1] to the `DataSourceExec` node, but can't push down `cost>50` because it requires the `ProjectionExec` node to be executed first: -/// -/// ```text -// ┌──────────────────────┐ -// │ CoalesceBatchesExec │ -// └──────────────────────┘ -// │ -// ▼ -// ┌──────────────────────┐ -// │ FilterExec │ -// │ filters = │ -// │ [cost>50] │ -// └──────────────────────┘ -// │ -// ▼ -// ┌──────────────────────┐ -// │ ProjectionExec │ -// │ cost = price * 1.2 │ -// └──────────────────────┘ -// │ -// ▼ -// ┌──────────────────────┐ -// │ DataSourceExec │ -// │ projection = * │ -// │ filters = [ id=1] │ -// └──────────────────────┘ -/// ``` -/// -/// There are also cases where we may be able to push down filters within a subtree but not the entire tree. -/// A good exmaple of this is aggreagation nodes: -/// -/// ```text -/// ┌──────────────────────┐ -/// │ ProjectionExec │ -/// │ projection = * │ -/// └──────────────────────┘ -/// │ -/// ▼ -/// ┌──────────────────────┐ -/// │ FilterExec │ -/// │ filters = [sum > 10] │ -/// └──────────────────────┘ -/// │ -/// ▼ -/// ┌───────────────────────┐ -/// │ AggregateExec │ -/// │ group by = [id] │ -/// │ aggregate = │ -/// │ [sum(price)] │ -/// └───────────────────────┘ -/// │ -/// ▼ -/// ┌──────────────────────┐ -/// │ FilterExec │ -/// │ filters = [id=1] │ -/// └──────────────────────┘ -/// │ -/// ▼ -/// ┌──────────────────────┐ -/// │ DataSourceExec │ -/// │ projection = * │ -/// └──────────────────────┘ -/// ``` -/// -/// The transformation here is to push down the `[id=1]` filter to the `DataSourceExec` node: -/// -/// ```text -/// ┌──────────────────────┐ -/// │ ProjectionExec │ -/// │ projection = * │ -/// └──────────────────────┘ -/// │ -/// ▼ -/// ┌──────────────────────┐ -/// │ FilterExec │ -/// │ filters = [sum > 10] │ -/// └──────────────────────┘ -/// │ -/// ▼ -/// ┌───────────────────────┐ -/// │ AggregateExec │ -/// │ group by = [id] │ -/// │ aggregate = │ -/// │ [sum(price)] │ -/// └───────────────────────┘ -/// │ -/// ▼ -/// ┌──────────────────────┐ -/// │ DataSourceExec │ -/// │ projection = * │ -/// │ filters = [id=1] │ -/// └──────────────────────┘ -/// ``` -/// -/// The point here is that: -/// 1. We cannot push down `sum > 10` through the `AggregateExec` node into the `DataSourceExec` node. -/// Any filters above the `AggregateExec` node are not pushed down. -/// This is determined by calling [`ExecutionPlan::filter_pushdown_request`] on the `AggregateExec` node. -/// 2. We need to keep recursing into the tree so that we can discover the other `FilterExec` node and push down the [id=1] filter. -/// -/// It is also possible to push down filters through joins and from joins. -/// For example, a hash join where we build a hash table of the left side and probe the right side -/// (ignoring why we would choose this order, typically it depends on the size of each table, etc.). -/// -/// ```text -/// ┌─────────────────────┐ -/// │ FilterExec │ -/// │ filters = │ -/// │ [d.size > 100] │ -/// └─────────────────────┘ -/// │ -/// │ -/// ┌──────────▼──────────┐ -/// │ │ -/// │ HashJoinExec │ -/// │ [u.dept@hash(d.id)] │ -/// │ │ -/// └─────────────────────┘ -/// │ -/// ┌────────────┴────────────┐ -/// ┌──────────▼──────────┐ ┌──────────▼──────────┐ -/// │ DataSourceExec │ │ DataSourceExec │ -/// │ alias [users as u] │ │ alias [dept as d] │ -/// │ │ │ │ -/// └─────────────────────┘ └─────────────────────┘ -/// ``` -/// -/// There are two pushdowns we can do here: -/// 1. Push down the `d.size > 100` filter through the `HashJoinExec` node to the `DataSourceExec` node for the `departments` table. -/// 2. Push down the hash table state from the `HashJoinExec` node to the `DataSourceExec` node to avoid reading -/// rows from teh `users` table that will be eliminated by the join. -/// This can be done via a bloom filter or similar. -/// -/// ```text -/// ┌─────────────────────┐ -/// │ │ -/// │ HashJoinExec │ -/// │ [u.dept@hash(d.id)] │ -/// │ │ -/// └─────────────────────┘ -/// │ -/// ┌────────────┴────────────┐ -/// ┌──────────▼──────────┐ ┌──────────▼──────────┐ -/// │ DataSourceExec │ │ DataSourceExec │ -/// │ alias [users as u] │ │ alias [dept as d] │ -/// │ filters = │ │ filters = │ -/// │ [depg@hash(d.id)] │ │ [ d.size > 100] │ -/// └─────────────────────┘ └─────────────────────┘ -/// ``` -/// -/// You may notice in this case that the filter is *dynamic*: the hash table is built -/// _after_ the `departments` table is read and at runtime. -/// We don't have a concrete `InList` filter or similar to push down at optimization time. -/// These sorts of dynamic filters are handled by building a specialized -/// [`PhysicalExpr`][datafusion_physical_expr::PhysicalExpr] that can be evaluated at runtime -/// and internally maintains a reference to the hash table or other state. -/// To make working with these sorts of dynamic filters more tractable we have the method `PhysicalExpr::snapshot` -/// (TODO: add reference after is merged) -/// which attempts to simplify a dynamic filter into a "basic" non-dynamic filter. -/// For a join this could mean converting it to an `InList` filter or a min/max filter for example. -/// See `datafusion/physical-plan/src/dynamic_filters.rs` for more details. -/// -/// Another form of dyanmic filter is pushing down the state of a `TopK` operator for queries like -/// `SELECT * FROM t ORDER BY id LIMIT 10`: -/// -/// ```text -/// ┌──────────────────────┐ -/// │ TopK │ -/// │ limit = 10 │ -/// │ order by = [id] │ -/// └──────────────────────┘ -/// │ -/// ▼ -/// ┌──────────────────────┐ -/// │ DataSourceExec │ -/// │ projection = * │ -/// └──────────────────────┘ -/// ``` -/// -/// We can avoid large amounts of data processing by transforming this into: -/// -/// ```text -/// ┌──────────────────────┐ -/// │ TopK │ -/// │ limit = 10 │ -/// │ order by = [id] │ -/// └──────────────────────┘ -/// │ -/// ▼ -/// ┌──────────────────────┐ -/// │ DataSourceExec │ -/// │ projection = * │ -/// │ filters = │ -/// │ [id < @ TopKHeap] │ -/// └──────────────────────┘ -/// ``` -/// -/// Now as we fill our `TopK` heap we can push down the state of the heap to the `DataSourceExec` node -/// to avoid reading files / row groups / pages / rows that could not possibly be in the top 10. -/// This is implemented in datafusion/physical-plan/src/sorts/sort_filters.rs. +/// See [`ExecutionPlan::try_pushdown_filters`] for a detailed description of the algorithm. #[derive(Debug)] pub struct PushdownFilter {} @@ -486,15 +47,23 @@ impl PhysicalOptimizerRule for PushdownFilter { plan: Arc, _config: &ConfigOptions, ) -> Result> { - if let Some(result) = pushdown_filters(&plan, &[])? { - Ok(result.inner) - } else { - Ok(plan) + match plan.try_pushdown_filters(&plan, &Vec::new())? { + ExecutionPlanFilterPushdownResult::NotPushed => Ok(plan), + ExecutionPlanFilterPushdownResult::Pushed { inner, support } => { + if !support.is_empty() { + return Err( + DataFusionError::Plan( + format!("PushdownFilter: plan returned support length does not match filters length: {} != 0", support.len() + )) + ); + } + Ok(inner) + } } } fn name(&self) -> &str { - "FilterPushdown" + "PushdownFilter" } fn schema_check(&self) -> bool { diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index 1ea6c14f4d2a7..5b549972846b6 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -16,6 +16,7 @@ // under the License. pub use crate::display::{DefaultDisplay, DisplayAs, DisplayFormatType, VerboseDisplay}; +use crate::filter_pushdown::{FilterPushdownResult, FilterPushdownSupport}; pub use crate::metrics::Metric; pub use crate::ordering::InputOrderMode; pub use crate::stream::EmptyRecordBatchStream; @@ -45,7 +46,7 @@ use crate::stream::RecordBatchStreamAdapter; use arrow::array::{Array, RecordBatch}; use arrow::datatypes::SchemaRef; use datafusion_common::config::ConfigOptions; -use datafusion_common::{exec_err, Constraints, Result}; +use datafusion_common::{exec_err, Constraints, DataFusionError, Result}; use datafusion_common_runtime::JoinSet; use datafusion_execution::TaskContext; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering, PhysicalExprRef}; @@ -468,105 +469,322 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { Ok(None) } - /// Returns a set of filters that this operator owns but would like to be pushed down. - /// - /// For example, a `TopK` operator may produce dynamic filters that - /// reference it's current state, while a `FilterExec` will just hand of the - /// filters it has as is. - /// - /// The default implementation returns an empty vector. These filters are - /// applied row-by row: - /// 1. any that return `false` or `NULL` will be filtered out - /// 2. any that return `true` will be kept. - /// - /// The expressions returned **must** always be Boolean ( `true`, `false` or - /// NULL); other truthy or falsy values are not allowed (e.g. `0`, `1`). - /// - /// # Returns - /// A vector of filters that this operator would like to push down. - /// These should be treated as the split conjunction of a `WHERE` clause. - /// That is, a query such as `WHERE a = 1 AND b = 2` would return two - /// filters: `a = 1` and `b = 2`. - /// They can be combined into a single filter using - /// [`conjunction`][datafusion_physical_expr::conjunction]. - fn filters_for_pushdown(&self) -> Result>> { - Ok(Vec::new()) - } - - /// Checks which filters this node allows to be pushed down through it from a parent to a child. - /// For example, a `ProjectionExec` node can allow filters that only reference - /// columns it did not create through but filters that reference columns it is creating cannot be pushed down any further. - /// That is, it only allows some filters through because it changes the schema of the data. - /// Aggregation nodes may not allow any filters to be pushed down as they change the cardinality of the data. - /// RepartitionExec nodes allow all filters to be pushed down as they don't change the schema or cardinality. - fn filter_pushdown_request( + /// A physical optimizer rule that pushes down filters in the execution plan. + /// For example, consider the following plan: + /// + /// ```text + /// ┌──────────────────────┐ + /// │ CoalesceBatchesExec │ + /// └──────────────────────┘ + /// │ + /// ▼ + /// ┌──────────────────────┐ + /// │ FilterExec │ + /// │ filters = [ id=1] │ + /// └──────────────────────┘ + /// │ + /// ▼ + /// ┌──────────────────────┐ + /// │ DataSourceExec │ + /// │ projection = * │ + /// └──────────────────────┘ + /// ``` + /// + /// Our goal is to move the `id = 1` filter from the `FilterExec` node to the `DataSourceExec` node. + /// If this filter is selective it can avoid massive amounts of data being read from the source (the projection is `*` so all matching columns are read). + /// In this simple case we: + /// 1. Enter the recursion with no filters. + /// 2. We find the `FilterExec` node and it tells us that it has a filter (see [`ExecutionPlan::filters_for_pushdown`] and `datafusion::physical_plan::filter::FilterExec`). + /// 3. We recurse down into it's children (the `DataSourceExec` node) now carrying the filters `[id = 1]`. + /// 4. The `DataSourceExec` node tells us that it can handle the filter and we mark it as handled exact (see [`ExecutionPlan::with_filter_pushdown_result`]). + /// 5. Since the `DataSourceExec` node has no children we recurse back up the tree. + /// 6. We now tell the `FilterExec` node that it has a child that can handle the filter and we mark it as handled exact (see [`ExecutionPlan::with_filter_pushdown_result`]). + /// The `FilterExec` node can now return a new execution plan, either a copy of itself without that filter or if has no work left to do it can even return the child node directly. + /// 7. We recurse back up to `CoalesceBatchesExec` and do nothing there since it had no filters to push down. + /// + /// The new plan looks like: + /// + /// ```text + /// ┌──────────────────────┐ + /// │ CoalesceBatchesExec │ + /// └──────────────────────┘ + /// │ + /// ▼ + /// ┌──────────────────────┐ + /// │ DataSourceExec │ + // │ projection = * │ + // │ filters = [ id=1] │ + /// └──────────────────────┘ + /// ``` + /// + /// Let's consider a more complex example involving a `ProjectionExec` node in betweeen the `FilterExec` and `DataSourceExec` nodes that creates a new column that the filter depends on. + /// + /// ```text + // ┌──────────────────────┐ + // │ CoalesceBatchesExec │ + // └──────────────────────┘ + // │ + // ▼ + // ┌──────────────────────┐ + // │ FilterExec │ + // │ filters = │ + // │ [cost>50,id=1] │ + // └──────────────────────┘ + // │ + // ▼ + // ┌──────────────────────┐ + // │ ProjectionExec │ + // │ cost = price * 1.2 │ + // └──────────────────────┘ + // │ + // ▼ + // ┌──────────────────────┐ + // │ DataSourceExec │ + // │ projection = * │ + // └──────────────────────┘ + /// ``` + /// + /// We want to push down the filters [id=1] to the `DataSourceExec` node, but can't push down `cost>50` because it requires the `ProjectionExec` node to be executed first: + /// + /// ```text + // ┌──────────────────────┐ + // │ CoalesceBatchesExec │ + // └──────────────────────┘ + // │ + // ▼ + // ┌──────────────────────┐ + // │ FilterExec │ + // │ filters = │ + // │ [cost>50] │ + // └──────────────────────┘ + // │ + // ▼ + // ┌──────────────────────┐ + // │ ProjectionExec │ + // │ cost = price * 1.2 │ + // └──────────────────────┘ + // │ + // ▼ + // ┌──────────────────────┐ + // │ DataSourceExec │ + // │ projection = * │ + // │ filters = [ id=1] │ + // └──────────────────────┘ + /// ``` + /// + /// There are also cases where we may be able to push down filters within a subtree but not the entire tree. + /// A good exmaple of this is aggreagation nodes: + /// + /// ```text + /// ┌──────────────────────┐ + /// │ ProjectionExec │ + /// │ projection = * │ + /// └──────────────────────┘ + /// │ + /// ▼ + /// ┌──────────────────────┐ + /// │ FilterExec │ + /// │ filters = [sum > 10] │ + /// └──────────────────────┘ + /// │ + /// ▼ + /// ┌───────────────────────┐ + /// │ AggregateExec │ + /// │ group by = [id] │ + /// │ aggregate = │ + /// │ [sum(price)] │ + /// └───────────────────────┘ + /// │ + /// ▼ + /// ┌──────────────────────┐ + /// │ FilterExec │ + /// │ filters = [id=1] │ + /// └──────────────────────┘ + /// │ + /// ▼ + /// ┌──────────────────────┐ + /// │ DataSourceExec │ + /// │ projection = * │ + /// └──────────────────────┘ + /// ``` + /// + /// The transformation here is to push down the `[id=1]` filter to the `DataSourceExec` node: + /// + /// ```text + /// ┌──────────────────────┐ + /// │ ProjectionExec │ + /// │ projection = * │ + /// └──────────────────────┘ + /// │ + /// ▼ + /// ┌──────────────────────┐ + /// │ FilterExec │ + /// │ filters = [sum > 10] │ + /// └──────────────────────┘ + /// │ + /// ▼ + /// ┌───────────────────────┐ + /// │ AggregateExec │ + /// │ group by = [id] │ + /// │ aggregate = │ + /// │ [sum(price)] │ + /// └───────────────────────┘ + /// │ + /// ▼ + /// ┌──────────────────────┐ + /// │ DataSourceExec │ + /// │ projection = * │ + /// │ filters = [id=1] │ + /// └──────────────────────┘ + /// ``` + /// + /// The point here is that: + /// 1. We cannot push down `sum > 10` through the `AggregateExec` node into the `DataSourceExec` node. + /// Any filters above the `AggregateExec` node are not pushed down. + /// This is determined by calling [`ExecutionPlan::filter_pushdown_request`] on the `AggregateExec` node. + /// 2. We need to keep recursing into the tree so that we can discover the other `FilterExec` node and push down the [id=1] filter. + /// + /// It is also possible to push down filters through joins and from joins. + /// For example, a hash join where we build a hash table of the left side and probe the right side + /// (ignoring why we would choose this order, typically it depends on the size of each table, etc.). + /// + /// ```text + /// ┌─────────────────────┐ + /// │ FilterExec │ + /// │ filters = │ + /// │ [d.size > 100] │ + /// └─────────────────────┘ + /// │ + /// │ + /// ┌──────────▼──────────┐ + /// │ │ + /// │ HashJoinExec │ + /// │ [u.dept@hash(d.id)] │ + /// │ │ + /// └─────────────────────┘ + /// │ + /// ┌────────────┴────────────┐ + /// ┌──────────▼──────────┐ ┌──────────▼──────────┐ + /// │ DataSourceExec │ │ DataSourceExec │ + /// │ alias [users as u] │ │ alias [dept as d] │ + /// │ │ │ │ + /// └─────────────────────┘ └─────────────────────┘ + /// ``` + /// + /// There are two pushdowns we can do here: + /// 1. Push down the `d.size > 100` filter through the `HashJoinExec` node to the `DataSourceExec` node for the `departments` table. + /// 2. Push down the hash table state from the `HashJoinExec` node to the `DataSourceExec` node to avoid reading + /// rows from teh `users` table that will be eliminated by the join. + /// This can be done via a bloom filter or similar. + /// + /// ```text + /// ┌─────────────────────┐ + /// │ │ + /// │ HashJoinExec │ + /// │ [u.dept@hash(d.id)] │ + /// │ │ + /// └─────────────────────┘ + /// │ + /// ┌────────────┴────────────┐ + /// ┌──────────▼──────────┐ ┌──────────▼──────────┐ + /// │ DataSourceExec │ │ DataSourceExec │ + /// │ alias [users as u] │ │ alias [dept as d] │ + /// │ filters = │ │ filters = │ + /// │ [depg@hash(d.id)] │ │ [ d.size > 100] │ + /// └─────────────────────┘ └─────────────────────┘ + /// ``` + /// + /// You may notice in this case that the filter is *dynamic*: the hash table is built + /// _after_ the `departments` table is read and at runtime. + /// We don't have a concrete `InList` filter or similar to push down at optimization time. + /// These sorts of dynamic filters are handled by building a specialized + /// [`PhysicalExpr`][datafusion_physical_expr::PhysicalExpr] that can be evaluated at runtime + /// and internally maintains a reference to the hash table or other state. + /// To make working with these sorts of dynamic filters more tractable we have the method `PhysicalExpr::snapshot` + /// (TODO: add reference after is merged) + /// which attempts to simplify a dynamic filter into a "basic" non-dynamic filter. + /// For a join this could mean converting it to an `InList` filter or a min/max filter for example. + /// See `datafusion/physical-plan/src/dynamic_filters.rs` for more details. + /// + /// Another form of dyanmic filter is pushing down the state of a `TopK` operator for queries like + /// `SELECT * FROM t ORDER BY id LIMIT 10`: + /// + /// ```text + /// ┌──────────────────────┐ + /// │ TopK │ + /// │ limit = 10 │ + /// │ order by = [id] │ + /// └──────────────────────┘ + /// │ + /// ▼ + /// ┌──────────────────────┐ + /// │ DataSourceExec │ + /// │ projection = * │ + /// └──────────────────────┘ + /// ``` + /// + /// We can avoid large amounts of data processing by transforming this into: + /// + /// ```text + /// ┌──────────────────────┐ + /// │ TopK │ + /// │ limit = 10 │ + /// │ order by = [id] │ + /// └──────────────────────┘ + /// │ + /// ▼ + /// ┌──────────────────────┐ + /// │ DataSourceExec │ + /// │ projection = * │ + /// │ filters = │ + /// │ [id < @ TopKHeap] │ + /// └──────────────────────┘ + /// ``` + /// + /// Now as we fill our `TopK` heap we can push down the state of the heap to the `DataSourceExec` node + /// to avoid reading files / row groups / pages / rows that could not possibly be in the top 10. + /// This is implemented in datafusion/physical-plan/src/sorts/sort_filters.rs. + fn try_pushdown_filters( &self, - filters: &[PhysicalExprRef], - ) -> Result> { - Ok(vec![FilterPushdownAllowed::Disallowed; filters.len()]) - } - - /// After we've attempted to push down filters into this node's children - /// this will be called with the result for each filter that this node gave in `filters_for_pushdown` - /// **and** any filters that children could not handle. - fn with_filter_pushdown_result( - self: Arc, - _own_filters_result: &[FilterSupport], - _parent_filters_remaining: &[PhysicalExprRef], - ) -> Result> { - Ok(None) - } -} - -/// The answer to the question: "Can this filter be pushed down through this plan?" -/// Note that this is different from [`FilterSupport`] which is the answer to "Can *this* plan handle this filter?" -#[derive(Debug, Clone)] -pub enum FilterPushdownAllowed { - /// The operator allows this filter to be pushed down to its children. - /// The operator may choose to return a *different* filter expression - /// that is equivalent to the original filter, e.g. to deal with column indexes in a projection - /// or because the original filter can't be pushed down as is but a less-selective filter can be. - Allowed(Arc), - /// The operator does not allow this filter to be pushed down to its children. - Disallowed, -} - -/// The answer to the question: "Can this operator handle this filter itself?" -/// Note that this is different from [`FilterPushdownAllowed`] which is the answer to "Can *this* plan handle this filter?" -#[derive(Debug, Clone, Copy)] -pub enum FilterSupport { - /// Filter may not have been pushed down to the child plan, or the child plan - /// can only partially apply the filter but may have false positives (but not false negatives). - /// In this case the parent **must** behave as if the filter was not pushed down - /// and must apply the filter itself. - Unhandled, - /// Filter was pushed down to the child plan and the child plan promises that - /// it will apply the filter correctly with no false positives or false negatives. - /// The parent can safely drop the filter. - HandledExact, -} - -/// The combined result of a filter pushdown operation. -/// This includes: -/// * The inner plan that was produced by the pushdown operation. -/// * The support for each filter that was pushed down. -pub struct FilterPushdownResult { - pub inner: T, - pub support: Vec, -} - -impl FilterPushdownResult { - pub fn new(plan: T, support: Vec) -> Self { - Self { - inner: plan, - support, + plan: &Arc, + parent_filters: &[PhysicalExprRef], + ) -> Result { + // By default assume that: + // * Parent filters can't be passed onto children. + // * We have no filters to contribute. + // But we still want to recurse into our children in case a subtree has pushdowns within + // it and thus we need to replace our children with the new plans. + let mut new_children = Vec::with_capacity(self.children().len()); + let mut pushed = false; + for child in self.children() { + match child.try_pushdown_filters(child, &Vec::new())? { + ExecutionPlanFilterPushdownResult::NotPushed => { + // No pushdown possible, keep this child as is + new_children.push(Arc::clone(child)); + } + ExecutionPlanFilterPushdownResult::Pushed { inner, support } => { + // We have a child that has pushed down some filters + new_children.push(inner); + pushed = true; + // Support should be empty, we didn't pass any filters + if !support.is_empty() { + return Err(DataFusionError::Internal( + "Child plan did not have any filters pushed down".to_string(), + )); + } + } + } + } + if pushed { + let new_inner = + with_new_children_if_necessary(Arc::clone(plan), new_children)?; + Ok(ExecutionPlanFilterPushdownResult::Pushed { + inner: new_inner, + support: vec![FilterPushdownSupport::Unsupported; parent_filters.len()], + }) + } else { + Ok(ExecutionPlanFilterPushdownResult::NotPushed) } - } - - pub fn is_exact(&self) -> bool { - self.support - .iter() - .all(|s| matches!(s, FilterSupport::HandledExact)) } } diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 6085680fc5a3f..b57fc05e69894 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -25,19 +25,17 @@ use super::{ RecordBatchStream, SendableRecordBatchStream, Statistics, }; use crate::common::can_project; -use crate::execution_plan::{ - CardinalityEffect, ExecutionPlanFilterPushdownResult, FilterPushdownAllowed, - FilterSupport, -}; +use crate::execution_plan::CardinalityEffect; +use crate::filter_pushdown::FilterPushdownSupport; use crate::projection::{ make_with_child, try_embed_projection, update_expr, EmbeddedProjection, ProjectionExec, }; +use crate::ExecutionPlanFilterPushdownResult; use crate::{ metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}, DisplayFormatType, ExecutionPlan, }; -use datafusion_physical_expr::expressions::lit; use arrow::compute::filter_record_batch; use arrow::datatypes::{DataType, SchemaRef}; @@ -54,7 +52,7 @@ use datafusion_physical_expr::expressions::BinaryExpr; use datafusion_physical_expr::intervals::utils::check_support; use datafusion_physical_expr::utils::{collect_columns, reassign_predicate_columns}; use datafusion_physical_expr::{ - analyze, conjunction_opt, split_conjunction, AcrossPartitions, AnalysisContext, + analyze, conjunction, split_conjunction, AcrossPartitions, AnalysisContext, ConstExpr, ExprBoundaries, PhysicalExpr, PhysicalExprRef, }; @@ -438,89 +436,75 @@ impl ExecutionPlan for FilterExec { try_embed_projection(projection, self) } - fn filter_pushdown_request( + fn try_pushdown_filters( &self, - filters: &[PhysicalExprRef], - ) -> Result> { - let filters = if self.projection.is_some() { + _plan: &Arc, + parent_filters: &[PhysicalExprRef], + ) -> Result { + let mut all_filters = parent_filters.to_vec(); + all_filters.push(Arc::clone(&self.predicate)); + let all_filters = if self.projection.is_some() { let input_schema = self.input.schema(); - filters - .iter() - .map(|f| reassign_predicate_columns(Arc::clone(f), &input_schema, false)) + all_filters + .into_iter() + .map(|f| reassign_predicate_columns(f, &input_schema, false)) .collect::>>()? } else { - filters.to_vec() + all_filters }; - - Ok(filters - .into_iter() - .map(FilterPushdownAllowed::Allowed) - .collect()) - } - - fn filters_for_pushdown(&self) -> Result>> { - let predicate = reassign_predicate_columns( - Arc::clone(&self.predicate), - &self.input.schema(), - false, - )?; - Ok(vec![predicate]) - } - - fn with_filter_pushdown_result( - self: Arc, - own_filters_result: &[FilterSupport], - parent_filters_remaining: &[PhysicalExprRef], - ) -> Result> { - // Only keep filters who's index maps to the pushdown result Unsupported - let filters_for_pushdown = self.filters_for_pushdown()?; - let new_filters = filters_for_pushdown - .iter() - .zip(own_filters_result.iter()) - .filter_map(|(f, p)| { - if matches!(p, FilterSupport::HandledExact) { - // Exact pushdown support means we keep discard filter - None - } else { - // Otherwise we still have to apply it - Some(Arc::clone(f)) + let (new_predicate, new_input) = match self + .input + .try_pushdown_filters(&self.input, &all_filters)? + { + ExecutionPlanFilterPushdownResult::NotPushed => { + if parent_filters.is_empty() { + return Ok(ExecutionPlanFilterPushdownResult::NotPushed); } - }) - // Combine that with any leftover filters from parents that our children couldn't handle - .chain(parent_filters_remaining.iter().map(Arc::clone)); - - let new_predicate = conjunction_opt(new_filters); + (conjunction(all_filters), Arc::clone(&self.input)) + } + ExecutionPlanFilterPushdownResult::Pushed { inner, support } => { + // Split out the filters that the child plan handled and the ones it did not + let unhandled_filters = all_filters + .into_iter() + .zip(support) + .filter_map(|(f, s)| { + if matches!(s, FilterPushdownSupport::Exact) { + None + } else { + Some(f) + } + }) + .collect::>(); + // If there are no unhandled filters and we have no projection, return the inner plan + if unhandled_filters.is_empty() && self.projection.is_none() { + return Ok(ExecutionPlanFilterPushdownResult::Pushed { + inner, + support: vec![FilterPushdownSupport::Exact; parent_filters.len()], + }); + } + let new_predicate = conjunction(unhandled_filters); + (new_predicate, inner) + } + }; - if new_predicate.is_none() && self.projection.is_none() { - // We can remove ourselves from the execution tree - Ok(Some(ExecutionPlanFilterPushdownResult::new( - Arc::clone(&self.input), - vec![FilterSupport::HandledExact; parent_filters_remaining.len()], - ))) - } else { - // Even if we don't have a predicate we keep ourselves in the tree for our projection. - let new_predicate = new_predicate.unwrap_or_else(|| lit(true)); - let cache = Self::compute_properties( - &self.input, - &new_predicate, - self.default_selectivity, - self.projection.as_ref(), - )?; - Ok(Some(ExecutionPlanFilterPushdownResult { - inner: Arc::new(Self { - predicate: new_predicate, - input: Arc::clone(&self.input), - metrics: self.metrics.clone(), - default_selectivity: self.default_selectivity, - cache, - projection: self.projection.clone(), - }), - support: vec![ - FilterSupport::HandledExact; - parent_filters_remaining.len() - ], - })) - } + let cache = Self::compute_properties( + &self.input, + &new_predicate, + self.default_selectivity, + self.projection.as_ref(), + )?; + let new_self = Self { + predicate: new_predicate, + input: new_input, + metrics: self.metrics.clone(), + default_selectivity: self.default_selectivity, + cache, + projection: self.projection.clone(), + }; + Ok(ExecutionPlanFilterPushdownResult::Pushed { + inner: Arc::new(new_self), + support: vec![FilterPushdownSupport::Exact; parent_filters.len()], + }) } } diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 04fbd06fabcde..64e3d6f39a18f 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -43,8 +43,10 @@ pub use crate::display::{DefaultDisplay, DisplayAs, DisplayFormatType, VerboseDi pub use crate::execution_plan::{ collect, collect_partitioned, displayable, execute_input_stream, execute_stream, execute_stream_partitioned, get_plan_string, with_new_children_if_necessary, - ExecutionPlan, ExecutionPlanProperties, PlanProperties, + ExecutionPlan, ExecutionPlanFilterPushdownResult, ExecutionPlanProperties, + PlanProperties, }; +pub use crate::filter_pushdown::FilterPushdownResult; pub use crate::metrics::Metric; pub use crate::ordering::InputOrderMode; pub use crate::stream::EmptyRecordBatchStream; @@ -66,6 +68,7 @@ pub mod empty; pub mod execution_plan; pub mod explain; pub mod filter; +pub mod filter_pushdown; pub mod joins; pub mod limit; pub mod memory; diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 9f00ab8bc83a6..ba2596551f1d5 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -237,7 +237,7 @@ physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after coalesce_batches SAME TEXT AS ABOVE physical_plan after OutputRequirements DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true physical_plan after LimitAggregation SAME TEXT AS ABOVE -physical_plan after FilterPushdown SAME TEXT AS ABOVE +physical_plan after PushdownFilter SAME TEXT AS ABOVE physical_plan after LimitPushdown SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after SanityCheckPlan SAME TEXT AS ABOVE @@ -314,7 +314,7 @@ physical_plan after OutputRequirements 01)GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] 02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] physical_plan after LimitAggregation SAME TEXT AS ABOVE -physical_plan after FilterPushdown SAME TEXT AS ABOVE +physical_plan after PushdownFilter SAME TEXT AS ABOVE physical_plan after LimitPushdown DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after SanityCheckPlan SAME TEXT AS ABOVE @@ -355,7 +355,7 @@ physical_plan after OutputRequirements 01)GlobalLimitExec: skip=0, fetch=10 02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet physical_plan after LimitAggregation SAME TEXT AS ABOVE -physical_plan after FilterPushdown SAME TEXT AS ABOVE +physical_plan after PushdownFilter SAME TEXT AS ABOVE physical_plan after LimitPushdown DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after SanityCheckPlan SAME TEXT AS ABOVE From 3fbf37927d6c2f39b4ab6f3afe1c716130fc8da1 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Sat, 5 Apr 2025 15:24:52 -0500 Subject: [PATCH 11/40] Add missing file --- .../physical-plan/src/filter_pushdown.rs | 56 +++++++++++++++++++ 1 file changed, 56 insertions(+) create mode 100644 datafusion/physical-plan/src/filter_pushdown.rs diff --git a/datafusion/physical-plan/src/filter_pushdown.rs b/datafusion/physical-plan/src/filter_pushdown.rs new file mode 100644 index 0000000000000..c338c06663d51 --- /dev/null +++ b/datafusion/physical-plan/src/filter_pushdown.rs @@ -0,0 +1,56 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +/// The answer to the question: "Can this operator handle this filter itself?" +/// Note that this is different from [`FilterPushdownAllowed`] which is the answer to "Can *this* plan handle this filter?" +#[derive(Debug, Clone, Copy)] +pub enum FilterPushdownSupport { + /// Filter may not have been pushed down to the child plan, or the child plan + /// can only partially apply the filter but may have false positives (but not false negatives). + /// In this case the parent **must** behave as if the filter was not pushed down + /// and must apply the filter itself. + Unsupported, + /// Filter was pushed down to the child plan and the child plan promises that + /// it will apply the filter correctly with no false positives or false negatives. + /// The parent can safely drop the filter. + Exact, +} + +/// The combined result of a filter pushdown operation. +/// This includes: +/// * The inner plan that was produced by the pushdown operation. +/// * The support for each filter that was pushed down. +pub enum FilterPushdownResult { + /// No pushdown was possible, keep this node as is in the tree. + NotPushed, + /// Pushed some or all filters into this node. + /// The caller should replace the node in the tree with the new one provided + /// and should transmit to parents the support for each filter. + Pushed { + /// The inner node that was produced by the pushdown operation. + inner: T, + /// The support for each filter that was pushed down. + support: Vec, + }, +} + +impl FilterPushdownResult { + /// Craete a new [`FilterPushdownResult`]. + pub fn new(inner: T, support: Vec) -> Self { + Self::Pushed { inner, support } + } +} From e6721d14a62316829aade3d7d362b8a4c7d7c586 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Sat, 5 Apr 2025 16:06:20 -0500 Subject: [PATCH 12/40] Add tests --- .../physical_optimizer/filter_pushdown.rs | 125 +++++++++++++++++- .../physical-plan/src/coalesce_batches.rs | 10 +- .../physical-plan/src/execution_plan.rs | 27 ++++ .../physical-plan/src/repartition/mod.rs | 10 +- 4 files changed, 168 insertions(+), 4 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs index b8dfa4d209fac..085f116263e0e 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs @@ -34,11 +34,20 @@ use datafusion_datasource::{ file_scan_config::FileScanConfig, file_stream::FileOpener, }; -use datafusion_physical_expr::{conjunction, PhysicalExprRef}; +use datafusion_expr::test::function_stub::count_udaf; +use datafusion_physical_expr::expressions::col; +use datafusion_physical_expr::{ + aggregate::AggregateExprBuilder, conjunction, Partitioning, PhysicalExprRef, +}; use datafusion_physical_expr_common::physical_expr::fmt_sql; use datafusion_physical_optimizer::filter_pushdown::PushdownFilter; use datafusion_physical_optimizer::PhysicalOptimizerRule; -use datafusion_physical_plan::filter::FilterExec; +use datafusion_physical_plan::{ + aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}, + coalesce_batches::CoalesceBatchesExec, + filter::FilterExec, + repartition::RepartitionExec, +}; use datafusion_physical_plan::{ displayable, filter_pushdown::FilterPushdownSupport, metrics::ExecutionPlanMetricsSet, DisplayFormatType, ExecutionPlan, @@ -210,6 +219,118 @@ fn test_filter_collapse() { ); } +#[test] +fn test_filter_with_projection() { + let scan = test_scan(FilterPushdownSupport::Exact); + let projection = vec![1, 0]; + let projected_schema = Arc::new(schema().project(&projection).unwrap()); + let predicate = col_lit_predicate("a", "foo", &projected_schema); + let plan = Arc::new( + FilterExec::try_new(predicate, scan) + .unwrap() + .with_projection(Some(projection)) + .unwrap(), + ); + // expect the predicate to be pushed down into the DataSource but the FilterExec to be kept for its projection + // the pushed down filters should have their indices adjusted + + insta::assert_snapshot!( + OptimizationTest::new(plan, PushdownFilter{}), + @r" + OptimizationTest: + input: + - FilterExec: a@1 = foo, projection=[b@1, a@0] + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test + output: + Ok: + - FilterExec: true, projection=[b@1, a@0] + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, predicate=a@0 = foo + ", + ); +} + +#[test] +fn test_push_down_through_transparent_nodes() { + // expect the predicate to be pushed down into the DataSource + let scan = test_scan(FilterPushdownSupport::Exact); + let coalesce = Arc::new(CoalesceBatchesExec::new(scan, 1)); + let predicate = col_lit_predicate("a", "foo", schema()); + let filter = Arc::new(FilterExec::try_new(predicate, coalesce).unwrap()); + let repartition = Arc::new( + RepartitionExec::try_new(filter, Partitioning::RoundRobinBatch(1)).unwrap(), + ); + let predicate = col_lit_predicate("a", "bar", schema()); + let plan = Arc::new(FilterExec::try_new(predicate, repartition).unwrap()); + + // expect the predicate to be pushed down into the DataSource + insta::assert_snapshot!( + OptimizationTest::new(plan, PushdownFilter{}), + @r" + OptimizationTest: + input: + - FilterExec: a@0 = bar + - RepartitionExec: partitioning=RoundRobinBatch(1), input_partitions=0 + - FilterExec: a@0 = foo + - CoalesceBatchesExec: target_batch_size=1 + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test + output: + Ok: + - RepartitionExec: partitioning=RoundRobinBatch(1), input_partitions=0 + - CoalesceBatchesExec: target_batch_size=1 + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, predicate=a@0 = bar AND a@0 = foo + " + ); +} + +#[test] +fn test_no_pushdown_through_aggregates() { + // expect the predicate to be pushed down into the DataSource + let scan = test_scan(FilterPushdownSupport::Exact); + let aggregate_expr = + vec![ + AggregateExprBuilder::new(count_udaf(), vec![col("a", schema()).unwrap()]) + .schema(Arc::clone(&schema())) + .alias("cnt") + .build() + .map(Arc::new) + .unwrap(), + ]; + let group_by = PhysicalGroupBy::new_single(vec![ + (col("a", schema()).unwrap(), "a".to_string()), + (col("b", schema()).unwrap(), "b".to_string()), + ]); + let aggregate = Arc::new( + AggregateExec::try_new( + AggregateMode::Final, + group_by, + aggregate_expr.clone(), + vec![None], + scan, + Arc::clone(schema()), + ) + .unwrap(), + ); + let predicate = col_lit_predicate("a", "foo", schema()); + let plan = Arc::new(FilterExec::try_new(predicate, aggregate).unwrap()); + + // expect the predicate to be pushed down into the DataSource + insta::assert_snapshot!( + OptimizationTest::new(plan, PushdownFilter{}), + @r" + OptimizationTest: + input: + - FilterExec: a@0 = foo + - AggregateExec: mode=Final, gby=[a@0 as a, b@1 as b], aggr=[cnt] + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test + output: + Ok: + - FilterExec: a@0 = foo + - AggregateExec: mode=Final, gby=[a@0 as a, b@1 as b], aggr=[cnt] + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, predicate=true + " + ); +} + /// Schema: /// a: String /// b: String diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index 5244038b9ae27..398c2c55b7c9a 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -34,7 +34,7 @@ use datafusion_common::Result; use datafusion_execution::TaskContext; use crate::coalesce::{BatchCoalescer, CoalescerState}; -use crate::execution_plan::CardinalityEffect; +use crate::execution_plan::{try_pushdown_filters_to_input, CardinalityEffect}; use futures::ready; use futures::stream::{Stream, StreamExt}; @@ -212,6 +212,14 @@ impl ExecutionPlan for CoalesceBatchesExec { fn cardinality_effect(&self) -> CardinalityEffect { CardinalityEffect::Equal } + + fn try_pushdown_filters( + &self, + plan: &Arc, + parent_filters: &[datafusion_physical_expr::PhysicalExprRef], + ) -> Result { + try_pushdown_filters_to_input(plan, &self.input, parent_filters) + } } /// Stream for [`CoalesceBatchesExec`]. See [`CoalesceBatchesExec`] for more details. diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index 5b549972846b6..bb37729a3c1e7 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -790,6 +790,33 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { pub type ExecutionPlanFilterPushdownResult = FilterPushdownResult>; +/// A default implementation of [`ExecutionPlan::try_pushdown_filters`] that +/// pushes down filters transparently to the input of the plan for plans that: +/// * Have a single input / child node. +/// * Support transparent filter pushdown (do not modify the cardinality or schema of the data). +/// * Do not have any filters of their own. +pub fn try_pushdown_filters_to_input( + plan: &Arc, + input: &Arc, + parent_filters: &[PhysicalExprRef], +) -> Result { + match input.try_pushdown_filters(input, parent_filters)? { + ExecutionPlanFilterPushdownResult::NotPushed => { + // No pushdown possible, keep this child as is + Ok(ExecutionPlanFilterPushdownResult::NotPushed) + } + ExecutionPlanFilterPushdownResult::Pushed { inner, support } => { + // We have a child that has pushed down some filters + let new_inner = + with_new_children_if_necessary(Arc::clone(plan), vec![inner])?; + Ok(ExecutionPlanFilterPushdownResult::Pushed { + inner: new_inner, + support, + }) + } + } +} + /// [`ExecutionPlan`] Invariant Level /// /// What set of assertions ([Invariant]s) holds for a particular `ExecutionPlan` diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index ebc751201378b..1ff26072717bb 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -29,7 +29,7 @@ use super::metrics::{self, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; use super::{ DisplayAs, ExecutionPlanProperties, RecordBatchStream, SendableRecordBatchStream, }; -use crate::execution_plan::CardinalityEffect; +use crate::execution_plan::{try_pushdown_filters_to_input, CardinalityEffect}; use crate::hash_utils::create_hashes; use crate::metrics::BaselineMetrics; use crate::projection::{all_columns, make_with_child, update_expr, ProjectionExec}; @@ -723,6 +723,14 @@ impl ExecutionPlan for RepartitionExec { new_partitioning, )?))) } + + fn try_pushdown_filters( + &self, + plan: &Arc, + parent_filters: &[datafusion_physical_expr::PhysicalExprRef], + ) -> Result { + try_pushdown_filters_to_input(plan, &self.input, parent_filters) + } } impl RepartitionExec { From b7b588bfdda98a459324e2f2ef071709a3fa2439 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Sat, 5 Apr 2025 16:11:27 -0500 Subject: [PATCH 13/40] pipe config in --- .../physical_optimizer/filter_pushdown.rs | 1 + datafusion/datasource/src/file.rs | 2 + datafusion/datasource/src/file_scan_config.rs | 7 +- datafusion/datasource/src/source.rs | 7 +- .../physical-optimizer/src/filter_pushdown.rs | 4 +- .../physical-plan/src/coalesce_batches.rs | 4 +- .../physical-plan/src/execution_plan.rs | 6 +- datafusion/physical-plan/src/filter.rs | 70 ++++++++++--------- .../physical-plan/src/repartition/mod.rs | 4 +- 9 files changed, 64 insertions(+), 41 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs index 085f116263e0e..0c937ea01658a 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs @@ -149,6 +149,7 @@ impl FileSource for TestSource { fn try_pushdown_filters( &self, filters: &[PhysicalExprRef], + _config: &ConfigOptions, ) -> Result { let new = Arc::new(TestSource { support: self.support, diff --git a/datafusion/datasource/src/file.rs b/datafusion/datasource/src/file.rs index 98420c4c88c68..45a05c7b4b322 100644 --- a/datafusion/datasource/src/file.rs +++ b/datafusion/datasource/src/file.rs @@ -26,6 +26,7 @@ use crate::file_groups::FileGroupPartitioner; use crate::file_scan_config::FileScanConfig; use crate::file_stream::FileOpener; use arrow::datatypes::SchemaRef; +use datafusion_common::config::ConfigOptions; use datafusion_common::{Result, Statistics}; use datafusion_physical_expr::{LexOrdering, PhysicalExprRef}; use datafusion_physical_plan::filter_pushdown::FilterPushdownResult; @@ -98,6 +99,7 @@ pub trait FileSource: Send + Sync { fn try_pushdown_filters( &self, _filters: &[PhysicalExprRef], + _config: &ConfigOptions, ) -> Result { Ok(FileSourceFilterPushdownResult::NotPushed) } diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index 82e23c7370adb..f8235e7cff673 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -31,7 +31,9 @@ use arrow::{ buffer::Buffer, datatypes::{ArrowNativeType, DataType, Field, Schema, SchemaRef, UInt16Type}, }; -use datafusion_common::{exec_err, ColumnStatistics, Constraints, Result, Statistics}; +use datafusion_common::{ + config::ConfigOptions, exec_err, ColumnStatistics, Constraints, Result, Statistics, +}; use datafusion_common::{DataFusionError, ScalarValue}; use datafusion_execution::{ object_store::ObjectStoreUrl, SendableRecordBatchStream, TaskContext, @@ -588,8 +590,9 @@ impl DataSource for FileScanConfig { fn try_pushdown_filters( &self, filters: &[PhysicalExprRef], + config: &ConfigOptions, ) -> Result { - match self.file_source.try_pushdown_filters(filters)? { + match self.file_source.try_pushdown_filters(filters, config)? { FileSourceFilterPushdownResult::NotPushed => { Ok(DataSourceFilterPushdownResult::NotPushed) } diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index 453762b7a656a..8beaf227fbf41 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -84,6 +84,7 @@ pub trait DataSource: Send + Sync + Debug { fn try_pushdown_filters( &self, _filters: &[PhysicalExprRef], + _config: &ConfigOptions, ) -> Result { Ok(DataSourceFilterPushdownResult::NotPushed) } @@ -207,8 +208,12 @@ impl ExecutionPlan for DataSourceExec { &self, _plan: &Arc, parent_filters: &[PhysicalExprRef], + config: &ConfigOptions, ) -> Result { - match self.data_source.try_pushdown_filters(parent_filters)? { + match self + .data_source + .try_pushdown_filters(parent_filters, config)? + { DataSourceFilterPushdownResult::NotPushed => { Ok(ExecutionPlanFilterPushdownResult::NotPushed) } diff --git a/datafusion/physical-optimizer/src/filter_pushdown.rs b/datafusion/physical-optimizer/src/filter_pushdown.rs index 4e2c9f3a9507f..c01f14e79c8d2 100644 --- a/datafusion/physical-optimizer/src/filter_pushdown.rs +++ b/datafusion/physical-optimizer/src/filter_pushdown.rs @@ -45,9 +45,9 @@ impl PhysicalOptimizerRule for PushdownFilter { fn optimize( &self, plan: Arc, - _config: &ConfigOptions, + config: &ConfigOptions, ) -> Result> { - match plan.try_pushdown_filters(&plan, &Vec::new())? { + match plan.try_pushdown_filters(&plan, &Vec::new(), config)? { ExecutionPlanFilterPushdownResult::NotPushed => Ok(plan), ExecutionPlanFilterPushdownResult::Pushed { inner, support } => { if !support.is_empty() { diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index 398c2c55b7c9a..77dabb72aa767 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -30,6 +30,7 @@ use crate::{ use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; +use datafusion_common::config::ConfigOptions; use datafusion_common::Result; use datafusion_execution::TaskContext; @@ -217,8 +218,9 @@ impl ExecutionPlan for CoalesceBatchesExec { &self, plan: &Arc, parent_filters: &[datafusion_physical_expr::PhysicalExprRef], + config: &ConfigOptions, ) -> Result { - try_pushdown_filters_to_input(plan, &self.input, parent_filters) + try_pushdown_filters_to_input(plan, &self.input, parent_filters, config) } } diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index bb37729a3c1e7..b4c71c8f76e78 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -748,6 +748,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { &self, plan: &Arc, parent_filters: &[PhysicalExprRef], + config: &ConfigOptions, ) -> Result { // By default assume that: // * Parent filters can't be passed onto children. @@ -757,7 +758,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { let mut new_children = Vec::with_capacity(self.children().len()); let mut pushed = false; for child in self.children() { - match child.try_pushdown_filters(child, &Vec::new())? { + match child.try_pushdown_filters(child, &Vec::new(), config)? { ExecutionPlanFilterPushdownResult::NotPushed => { // No pushdown possible, keep this child as is new_children.push(Arc::clone(child)); @@ -799,8 +800,9 @@ pub fn try_pushdown_filters_to_input( plan: &Arc, input: &Arc, parent_filters: &[PhysicalExprRef], + config: &ConfigOptions, ) -> Result { - match input.try_pushdown_filters(input, parent_filters)? { + match input.try_pushdown_filters(input, parent_filters, config)? { ExecutionPlanFilterPushdownResult::NotPushed => { // No pushdown possible, keep this child as is Ok(ExecutionPlanFilterPushdownResult::NotPushed) diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index b57fc05e69894..e542b383bbd2b 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -41,6 +41,7 @@ use arrow::compute::filter_record_batch; use arrow::datatypes::{DataType, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_common::cast::as_boolean_array; +use datafusion_common::config::ConfigOptions; use datafusion_common::stats::Precision; use datafusion_common::{ internal_err, plan_err, project_schema, DataFusionError, Result, ScalarValue, @@ -440,6 +441,7 @@ impl ExecutionPlan for FilterExec { &self, _plan: &Arc, parent_filters: &[PhysicalExprRef], + config: &ConfigOptions, ) -> Result { let mut all_filters = parent_filters.to_vec(); all_filters.push(Arc::clone(&self.predicate)); @@ -452,40 +454,44 @@ impl ExecutionPlan for FilterExec { } else { all_filters }; - let (new_predicate, new_input) = match self - .input - .try_pushdown_filters(&self.input, &all_filters)? - { - ExecutionPlanFilterPushdownResult::NotPushed => { - if parent_filters.is_empty() { - return Ok(ExecutionPlanFilterPushdownResult::NotPushed); + let (new_predicate, new_input) = + match self + .input + .try_pushdown_filters(&self.input, &all_filters, config)? + { + ExecutionPlanFilterPushdownResult::NotPushed => { + if parent_filters.is_empty() { + return Ok(ExecutionPlanFilterPushdownResult::NotPushed); + } + (conjunction(all_filters), Arc::clone(&self.input)) } - (conjunction(all_filters), Arc::clone(&self.input)) - } - ExecutionPlanFilterPushdownResult::Pushed { inner, support } => { - // Split out the filters that the child plan handled and the ones it did not - let unhandled_filters = all_filters - .into_iter() - .zip(support) - .filter_map(|(f, s)| { - if matches!(s, FilterPushdownSupport::Exact) { - None - } else { - Some(f) - } - }) - .collect::>(); - // If there are no unhandled filters and we have no projection, return the inner plan - if unhandled_filters.is_empty() && self.projection.is_none() { - return Ok(ExecutionPlanFilterPushdownResult::Pushed { - inner, - support: vec![FilterPushdownSupport::Exact; parent_filters.len()], - }); + ExecutionPlanFilterPushdownResult::Pushed { inner, support } => { + // Split out the filters that the child plan handled and the ones it did not + let unhandled_filters = all_filters + .into_iter() + .zip(support) + .filter_map(|(f, s)| { + if matches!(s, FilterPushdownSupport::Exact) { + None + } else { + Some(f) + } + }) + .collect::>(); + // If there are no unhandled filters and we have no projection, return the inner plan + if unhandled_filters.is_empty() && self.projection.is_none() { + return Ok(ExecutionPlanFilterPushdownResult::Pushed { + inner, + support: vec![ + FilterPushdownSupport::Exact; + parent_filters.len() + ], + }); + } + let new_predicate = conjunction(unhandled_filters); + (new_predicate, inner) } - let new_predicate = conjunction(unhandled_filters); - (new_predicate, inner) - } - }; + }; let cache = Self::compute_properties( &self.input, diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 1ff26072717bb..53d53900af2d6 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -43,6 +43,7 @@ use crate::{DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, Stat use arrow::array::{PrimitiveArray, RecordBatch, RecordBatchOptions}; use arrow::compute::take_arrays; use arrow::datatypes::{SchemaRef, UInt32Type}; +use datafusion_common::config::ConfigOptions; use datafusion_common::utils::transpose; use datafusion_common::HashMap; use datafusion_common::{not_impl_err, DataFusionError, Result}; @@ -728,8 +729,9 @@ impl ExecutionPlan for RepartitionExec { &self, plan: &Arc, parent_filters: &[datafusion_physical_expr::PhysicalExprRef], + config: &ConfigOptions, ) -> Result { - try_pushdown_filters_to_input(plan, &self.input, parent_filters) + try_pushdown_filters_to_input(plan, &self.input, parent_filters, config) } } From d1f01dd4026310883a8c7a6c8d4122659aa19623 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Sat, 5 Apr 2025 16:22:11 -0500 Subject: [PATCH 14/40] docstrings --- datafusion/datasource/src/file.rs | 5 ++++- datafusion/datasource/src/source.rs | 3 ++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/datafusion/datasource/src/file.rs b/datafusion/datasource/src/file.rs index 45a05c7b4b322..af0c68607edfd 100644 --- a/datafusion/datasource/src/file.rs +++ b/datafusion/datasource/src/file.rs @@ -95,7 +95,10 @@ pub trait FileSource: Send + Sync { } Ok(None) } - + /// Try to push down filters into this FileSource. + /// See [`ExecutionPlan::try_pushdown_filters`] for more details. + /// + /// [`ExecutionPlan::try_pushdown_filters`]: datafusion_physical_plan::ExecutionPlan::try_pushdown_filters fn try_pushdown_filters( &self, _filters: &[PhysicalExprRef], diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index 8beaf227fbf41..9df818c941a0a 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -80,7 +80,8 @@ pub trait DataSource: Send + Sync + Debug { &self, _projection: &ProjectionExec, ) -> Result>>; - + /// Try to push down filters into this DataSource. + /// See [`ExecutionPlan::try_pushdown_filters`] for more details. fn try_pushdown_filters( &self, _filters: &[PhysicalExprRef], From 5929d03138880259a2b5940663081767a1710ea1 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Sat, 5 Apr 2025 16:22:24 -0500 Subject: [PATCH 15/40] Update datafusion/physical-plan/src/filter_pushdown.rs --- datafusion/physical-plan/src/filter_pushdown.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/filter_pushdown.rs b/datafusion/physical-plan/src/filter_pushdown.rs index c338c06663d51..177b3725d574c 100644 --- a/datafusion/physical-plan/src/filter_pushdown.rs +++ b/datafusion/physical-plan/src/filter_pushdown.rs @@ -49,7 +49,7 @@ pub enum FilterPushdownResult { } impl FilterPushdownResult { - /// Craete a new [`FilterPushdownResult`]. + /// Create a new [`FilterPushdownResult`]. pub fn new(inner: T, support: Vec) -> Self { Self::Pushed { inner, support } } From 24483bcd7aa026a84a2c5947648c78b92f42bd8a Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Sat, 5 Apr 2025 16:35:07 -0500 Subject: [PATCH 16/40] fix --- .../physical-plan/src/execution_plan.rs | 55 ++++++++++++++----- .../physical-plan/src/filter_pushdown.rs | 11 +++- 2 files changed, 50 insertions(+), 16 deletions(-) diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index b4c71c8f76e78..13279a121d213 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -490,17 +490,18 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// └──────────────────────┘ /// ``` /// - /// Our goal is to move the `id = 1` filter from the `FilterExec` node to the `DataSourceExec` node. + /// Our goal is to move the `id = 1` filter from the [`FilterExec`] node to the `DataSourceExec` node. /// If this filter is selective it can avoid massive amounts of data being read from the source (the projection is `*` so all matching columns are read). /// In this simple case we: /// 1. Enter the recursion with no filters. - /// 2. We find the `FilterExec` node and it tells us that it has a filter (see [`ExecutionPlan::filters_for_pushdown`] and `datafusion::physical_plan::filter::FilterExec`). - /// 3. We recurse down into it's children (the `DataSourceExec` node) now carrying the filters `[id = 1]`. - /// 4. The `DataSourceExec` node tells us that it can handle the filter and we mark it as handled exact (see [`ExecutionPlan::with_filter_pushdown_result`]). - /// 5. Since the `DataSourceExec` node has no children we recurse back up the tree. - /// 6. We now tell the `FilterExec` node that it has a child that can handle the filter and we mark it as handled exact (see [`ExecutionPlan::with_filter_pushdown_result`]). - /// The `FilterExec` node can now return a new execution plan, either a copy of itself without that filter or if has no work left to do it can even return the child node directly. - /// 7. We recurse back up to `CoalesceBatchesExec` and do nothing there since it had no filters to push down. + /// 2. We find the [`FilterExec`] node and call [`ExecutionPlan::try_pushdown_filters`] on it. + /// 3. The [`FilterExec`] node tries to push it's filters + the filters from the parent nodes (in this case empty) + /// down into it's input, which is the `DataSourceExec` node. + /// 4. The `DataSourceExec` node accepts the filter and returns a [`FilterPushdownResult`] with a new copy of itself + /// and [`FilterPushdownSupport::Exact`] to indicate that the filter was pushed down and the caller no longer + /// needs to handle it. + /// 5. The [`FilterExec`] seeing that all filters were pushed down returns a [`FilterPushdownResult`] that directly + /// returns the new `DataSourceExec` node, effectively removing the [`FilterExec`] node from the plan. /// /// The new plan looks like: /// @@ -517,7 +518,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// └──────────────────────┘ /// ``` /// - /// Let's consider a more complex example involving a `ProjectionExec` node in betweeen the `FilterExec` and `DataSourceExec` nodes that creates a new column that the filter depends on. + /// Let's consider a more complex example involving a [`ProjectionExec`] node in betweeen the [`FilterExec`] and `DataSourceExec` nodes that creates a new column that the filter depends on. /// /// ```text // ┌──────────────────────┐ @@ -544,7 +545,8 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { // └──────────────────────┘ /// ``` /// - /// We want to push down the filters [id=1] to the `DataSourceExec` node, but can't push down `cost>50` because it requires the `ProjectionExec` node to be executed first: + /// We want to push down the filters [id=1] to the `DataSourceExec` node, but can't push down `cost>50` because it requires the [`ProjectionExec`] node to be executed first. + /// A simple thing to do would be to split up the filter into two separate filters and push down the first one: /// /// ```text // ┌──────────────────────┐ @@ -572,6 +574,28 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { // └──────────────────────┘ /// ``` /// + /// We can actually however do better by pushing down `price * 1.2 > 50` instead of `cost > 50`: + /// + /// ```text + /// ┌──────────────────────┐ + /// │ CoalesceBatchesExec │ + /// └──────────────────────┘ + /// │ + /// ▼ + /// ┌──────────────────────┐ + /// │ ProjectionExec │ + /// │ cost = price * 1.2 │ + /// └──────────────────────┘ + /// │ + /// ▼ + /// ┌──────────────────────┐ + /// │ DataSourceExec │ + /// │ projection = * │ + /// │ filters = [id=1, │ + /// │ price * 1.2 > 50] │ + /// └──────────────────────┘ + /// ``` + /// /// There are also cases where we may be able to push down filters within a subtree but not the entire tree. /// A good exmaple of this is aggreagation nodes: /// @@ -608,7 +632,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// └──────────────────────┘ /// ``` /// - /// The transformation here is to push down the `[id=1]` filter to the `DataSourceExec` node: + /// The transformation here is to push down the `id=1` filter to the `DataSourceExec` node: /// /// ```text /// ┌──────────────────────┐ @@ -642,7 +666,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// 1. We cannot push down `sum > 10` through the `AggregateExec` node into the `DataSourceExec` node. /// Any filters above the `AggregateExec` node are not pushed down. /// This is determined by calling [`ExecutionPlan::filter_pushdown_request`] on the `AggregateExec` node. - /// 2. We need to keep recursing into the tree so that we can discover the other `FilterExec` node and push down the [id=1] filter. + /// 2. We need to keep recursing into the tree so that we can discover the other [`FilterExec`] node and push down the `id=1` filter. /// /// It is also possible to push down filters through joins and from joins. /// For example, a hash join where we build a hash table of the left side and probe the right side @@ -698,7 +722,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// _after_ the `departments` table is read and at runtime. /// We don't have a concrete `InList` filter or similar to push down at optimization time. /// These sorts of dynamic filters are handled by building a specialized - /// [`PhysicalExpr`][datafusion_physical_expr::PhysicalExpr] that can be evaluated at runtime + /// [`PhysicalExpr`] that can be evaluated at runtime /// and internally maintains a reference to the hash table or other state. /// To make working with these sorts of dynamic filters more tractable we have the method `PhysicalExpr::snapshot` /// (TODO: add reference after is merged) @@ -744,6 +768,9 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// Now as we fill our `TopK` heap we can push down the state of the heap to the `DataSourceExec` node /// to avoid reading files / row groups / pages / rows that could not possibly be in the top 10. /// This is implemented in datafusion/physical-plan/src/sorts/sort_filters.rs. + /// + /// [`FilterExec`]: crate::filter::FilterExec + /// [`ProjectionExec`]: crate::projection::ProjectionExec fn try_pushdown_filters( &self, plan: &Arc, @@ -869,7 +896,7 @@ pub trait ExecutionPlanProperties { /// If this ExecutionPlan makes no changes to the schema of the rows flowing /// through it or how columns within each row relate to each other, it /// should return the equivalence properties of its input. For - /// example, since `FilterExec` may remove rows from its input, but does not + /// example, since [`FilterExec`] may remove rows from its input, but does not /// otherwise modify them, it preserves its input equivalence properties. /// However, since `ProjectionExec` may calculate derived expressions, it /// needs special handling. diff --git a/datafusion/physical-plan/src/filter_pushdown.rs b/datafusion/physical-plan/src/filter_pushdown.rs index 177b3725d574c..d2c60e5d2d9fd 100644 --- a/datafusion/physical-plan/src/filter_pushdown.rs +++ b/datafusion/physical-plan/src/filter_pushdown.rs @@ -15,8 +15,15 @@ // specific language governing permissions and limitations // under the License. -/// The answer to the question: "Can this operator handle this filter itself?" -/// Note that this is different from [`FilterPushdownAllowed`] which is the answer to "Can *this* plan handle this filter?" +/// Result of trying to push down fitlers to a child plan. +/// This is used by [`FilterPushdownResult`] to indicate whether the filter was +/// "absorbed" by the child ([`FilterPushdownSupport::Exact`]) or not +/// ([`FilterPushdownSupport::Unsupported`]). +/// If the filter was not absorbed, the parent plan must apply the filter +/// itself, or return to the caller that it was not pushed down. +/// If the filter was absorbed, the parent plan can drop the filter or +/// tell the caller that it was pushed down by forwarding on the [`FilterPushdownSupport::Exact`] +/// information. #[derive(Debug, Clone, Copy)] pub enum FilterPushdownSupport { /// Filter may not have been pushed down to the child plan, or the child plan From d0295ed0a942f376044e38b60d7460a8e2f65c51 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Sat, 5 Apr 2025 19:05:32 -0500 Subject: [PATCH 17/40] fix --- .../physical_optimizer/filter_pushdown.rs | 21 +++++++++++++------ .../physical-plan/src/execution_plan.rs | 4 +++- 2 files changed, 18 insertions(+), 7 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs index 0c937ea01658a..5597e1b6da69e 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs @@ -285,12 +285,20 @@ fn test_push_down_through_transparent_nodes() { #[test] fn test_no_pushdown_through_aggregates() { - // expect the predicate to be pushed down into the DataSource + // There are 2 important points here: + // 1. The outer filter is not pushed down into the aggregate because we haven't + // implemented that yet. + // 2. The inner filter **is** pushed down into the DataSource. let scan = test_scan(FilterPushdownSupport::Exact); + let filter = Arc::new(FilterExec::try_new( + col_lit_predicate("a", "foo", schema()), + scan.clone(), + ) + .unwrap()); let aggregate_expr = vec![ AggregateExprBuilder::new(count_udaf(), vec![col("a", schema()).unwrap()]) - .schema(Arc::clone(&schema())) + .schema(Arc::clone(schema())) .alias("cnt") .build() .map(Arc::new) @@ -306,7 +314,7 @@ fn test_no_pushdown_through_aggregates() { group_by, aggregate_expr.clone(), vec![None], - scan, + filter, Arc::clone(schema()), ) .unwrap(), @@ -321,13 +329,14 @@ fn test_no_pushdown_through_aggregates() { OptimizationTest: input: - FilterExec: a@0 = foo - - AggregateExec: mode=Final, gby=[a@0 as a, b@1 as b], aggr=[cnt] - - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test + - AggregateExec: mode=Final, gby=[a@0 as a, b@1 as b], aggr=[cnt], ordering_mode=PartiallySorted([0]) + - FilterExec: a@0 = foo + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test output: Ok: - FilterExec: a@0 = foo - AggregateExec: mode=Final, gby=[a@0 as a, b@1 as b], aggr=[cnt] - - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, predicate=true + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, predicate=a@0 = foo " ); } diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index 13279a121d213..ca2d102eec7a0 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -665,7 +665,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// The point here is that: /// 1. We cannot push down `sum > 10` through the `AggregateExec` node into the `DataSourceExec` node. /// Any filters above the `AggregateExec` node are not pushed down. - /// This is determined by calling [`ExecutionPlan::filter_pushdown_request`] on the `AggregateExec` node. + /// This is determined by calling [`ExecutionPlan::try_pushdown_filters`] on the [`AggregateExec`] node. /// 2. We need to keep recursing into the tree so that we can discover the other [`FilterExec`] node and push down the `id=1` filter. /// /// It is also possible to push down filters through joins and from joins. @@ -903,6 +903,8 @@ pub trait ExecutionPlanProperties { /// /// See also [`ExecutionPlan::maintains_input_order`] and [`Self::output_ordering`] /// for related concepts. + /// + /// [`FilterExec`]: crate::filter::FilterExec fn equivalence_properties(&self) -> &EquivalenceProperties; } From 2d462890861cd4989c4508959a48120fe837564a Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Sat, 5 Apr 2025 19:09:28 -0500 Subject: [PATCH 18/40] fmt --- .../core/tests/physical_optimizer/filter_pushdown.rs | 9 ++++----- datafusion/physical-plan/src/execution_plan.rs | 2 +- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs index 5597e1b6da69e..1e9167af290f3 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs @@ -290,11 +290,10 @@ fn test_no_pushdown_through_aggregates() { // implemented that yet. // 2. The inner filter **is** pushed down into the DataSource. let scan = test_scan(FilterPushdownSupport::Exact); - let filter = Arc::new(FilterExec::try_new( - col_lit_predicate("a", "foo", schema()), - scan.clone(), - ) - .unwrap()); + let filter = Arc::new( + FilterExec::try_new(col_lit_predicate("a", "foo", schema()), scan.clone()) + .unwrap(), + ); let aggregate_expr = vec![ AggregateExprBuilder::new(count_udaf(), vec![col("a", schema()).unwrap()]) diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index ca2d102eec7a0..abbb9e7aeeb49 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -903,7 +903,7 @@ pub trait ExecutionPlanProperties { /// /// See also [`ExecutionPlan::maintains_input_order`] and [`Self::output_ordering`] /// for related concepts. - /// + /// /// [`FilterExec`]: crate::filter::FilterExec fn equivalence_properties(&self) -> &EquivalenceProperties; } From 4318267a0cba6a6a2d086191424086c3d2f23d33 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Sun, 6 Apr 2025 00:45:48 -0500 Subject: [PATCH 19/40] fix doc --- datafusion/physical-plan/src/execution_plan.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index abbb9e7aeeb49..50724d7564c07 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -771,6 +771,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// /// [`FilterExec`]: crate::filter::FilterExec /// [`ProjectionExec`]: crate::projection::ProjectionExec + /// [`AggregateExec`]: crate::aggregates::AggregateExec fn try_pushdown_filters( &self, plan: &Arc, From 7d29056ab10fafcce80914a421b378dca05616bf Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Sun, 6 Apr 2025 08:19:59 -0500 Subject: [PATCH 20/40] add example usage of config --- .../physical_optimizer/filter_pushdown.rs | 65 +++++++++++++++++-- 1 file changed, 60 insertions(+), 5 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs index 1e9167af290f3..4e36151c8f262 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs @@ -60,9 +60,9 @@ use std::{ }; /// A placeholder data source that accepts filter pushdown -#[derive(Clone)] +#[derive(Clone, Default)] struct TestSource { - support: FilterPushdownSupport, + support: Option, predicate: Option, statistics: Option, } @@ -70,7 +70,7 @@ struct TestSource { impl TestSource { fn new(support: FilterPushdownSupport) -> Self { Self { - support, + support: Some(support), predicate: None, statistics: None, } @@ -149,8 +149,18 @@ impl FileSource for TestSource { fn try_pushdown_filters( &self, filters: &[PhysicalExprRef], - _config: &ConfigOptions, + config: &ConfigOptions, ) -> Result { + let support = match self.support { + Some(support) => support, + None => { + if config.execution.parquet.pushdown_filters { + FilterPushdownSupport::Exact + } else { + FilterPushdownSupport::Unsupported + } + } + }; let new = Arc::new(TestSource { support: self.support, predicate: Some(conjunction(filters.iter().map(Arc::clone))), @@ -158,7 +168,7 @@ impl FileSource for TestSource { }); Ok(FileSourceFilterPushdownResult::new( new, - vec![self.support; filters.len()], + vec![support; filters.len()], )) } } @@ -196,6 +206,51 @@ fn test_pushdown_into_scan() { ); } +/// Show that we can use config options to determine how to do pushdown. +#[test] +fn test_pushdown_into_scan_with_config_options() { + let scan = test_scan(FilterPushdownSupport::Exact); + let predicate = col_lit_predicate("a", "foo", schema()); + let plan = Arc::new(FilterExec::try_new(predicate, scan).unwrap()) as _; + + let mut cfg = ConfigOptions::default(); + cfg.execution.parquet.pushdown_filters = false; + insta::assert_snapshot!( + OptimizationTest::new_with_config( + Arc::clone(&plan), + PushdownFilter {}, + &cfg + ), + @r" + OptimizationTest: + input: + - FilterExec: a@0 = foo + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test + output: + Ok: + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, predicate=a@0 = foo + " + ); + + cfg.execution.parquet.pushdown_filters = true; + insta::assert_snapshot!( + OptimizationTest::new_with_config( + plan, + PushdownFilter {}, + &cfg + ), + @r" + OptimizationTest: + input: + - FilterExec: a@0 = foo + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test + output: + Ok: + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, predicate=a@0 = foo + " + ); +} + #[test] fn test_filter_collapse() { // filter should be pushed down into the parquet scan with two filters From d382bd394bfb5bd0796f3ffcc842eb327e44ee2e Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Sun, 6 Apr 2025 08:23:13 -0500 Subject: [PATCH 21/40] fix test --- .../physical_optimizer/filter_pushdown.rs | 21 ++++++++++--------- 1 file changed, 11 insertions(+), 10 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs index 4e36151c8f262..7c9988cd2688e 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs @@ -68,9 +68,9 @@ struct TestSource { } impl TestSource { - fn new(support: FilterPushdownSupport) -> Self { + fn new(support: Option) -> Self { Self { - support: Some(support), + support, predicate: None, statistics: None, } @@ -173,7 +173,7 @@ impl FileSource for TestSource { } } -fn test_scan(support: FilterPushdownSupport) -> Arc { +fn test_scan(support: Option) -> Arc { let schema = schema(); let source = Arc::new(TestSource::new(support)); let base_config = FileScanConfigBuilder::new( @@ -187,7 +187,7 @@ fn test_scan(support: FilterPushdownSupport) -> Arc { #[test] fn test_pushdown_into_scan() { - let scan = test_scan(FilterPushdownSupport::Exact); + let scan = test_scan(Some(FilterPushdownSupport::Exact)); let predicate = col_lit_predicate("a", "foo", schema()); let plan = Arc::new(FilterExec::try_new(predicate, scan).unwrap()); @@ -209,7 +209,7 @@ fn test_pushdown_into_scan() { /// Show that we can use config options to determine how to do pushdown. #[test] fn test_pushdown_into_scan_with_config_options() { - let scan = test_scan(FilterPushdownSupport::Exact); + let scan = test_scan(None); let predicate = col_lit_predicate("a", "foo", schema()); let plan = Arc::new(FilterExec::try_new(predicate, scan).unwrap()) as _; @@ -228,7 +228,8 @@ fn test_pushdown_into_scan_with_config_options() { - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test output: Ok: - - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, predicate=a@0 = foo + - FilterExec: a@0 = foo + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, predicate=a@0 = foo " ); @@ -254,7 +255,7 @@ fn test_pushdown_into_scan_with_config_options() { #[test] fn test_filter_collapse() { // filter should be pushed down into the parquet scan with two filters - let scan = test_scan(FilterPushdownSupport::Exact); + let scan = test_scan(Some(FilterPushdownSupport::Exact)); let predicate1 = col_lit_predicate("a", "foo", schema()); let filter1 = Arc::new(FilterExec::try_new(predicate1, scan).unwrap()); let predicate2 = col_lit_predicate("b", "bar", schema()); @@ -277,7 +278,7 @@ fn test_filter_collapse() { #[test] fn test_filter_with_projection() { - let scan = test_scan(FilterPushdownSupport::Exact); + let scan = test_scan(Some(FilterPushdownSupport::Exact)); let projection = vec![1, 0]; let projected_schema = Arc::new(schema().project(&projection).unwrap()); let predicate = col_lit_predicate("a", "foo", &projected_schema); @@ -308,7 +309,7 @@ fn test_filter_with_projection() { #[test] fn test_push_down_through_transparent_nodes() { // expect the predicate to be pushed down into the DataSource - let scan = test_scan(FilterPushdownSupport::Exact); + let scan = test_scan(Some(FilterPushdownSupport::Exact)); let coalesce = Arc::new(CoalesceBatchesExec::new(scan, 1)); let predicate = col_lit_predicate("a", "foo", schema()); let filter = Arc::new(FilterExec::try_new(predicate, coalesce).unwrap()); @@ -344,7 +345,7 @@ fn test_no_pushdown_through_aggregates() { // 1. The outer filter is not pushed down into the aggregate because we haven't // implemented that yet. // 2. The inner filter **is** pushed down into the DataSource. - let scan = test_scan(FilterPushdownSupport::Exact); + let scan = test_scan(Some(FilterPushdownSupport::Exact)); let filter = Arc::new( FilterExec::try_new(col_lit_predicate("a", "foo", schema()), scan.clone()) .unwrap(), From 2dfa8b803f2103c6ff81cfa483dbb70150feeb67 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Mon, 14 Apr 2025 13:57:21 +0300 Subject: [PATCH 22/40] convert exec API and optimizer rule --- .../physical_optimizer/filter_pushdown.rs | 66 +-- datafusion/datasource/src/file.rs | 14 +- datafusion/datasource/src/file_scan_config.rs | 53 +-- datafusion/datasource/src/source.rs | 48 +-- .../physical-optimizer/src/filter_pushdown.rs | 64 ++- .../physical-plan/src/coalesce_batches.rs | 20 +- .../physical-plan/src/execution_plan.rs | 383 +----------------- datafusion/physical-plan/src/filter.rs | 86 +--- .../physical-plan/src/filter_pushdown.rs | 70 ++-- datafusion/physical-plan/src/lib.rs | 4 +- .../physical-plan/src/repartition/mod.rs | 18 +- 11 files changed, 219 insertions(+), 607 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs index 7c9988cd2688e..ce1eed2a13588 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs @@ -42,15 +42,18 @@ use datafusion_physical_expr::{ use datafusion_physical_expr_common::physical_expr::fmt_sql; use datafusion_physical_optimizer::filter_pushdown::PushdownFilter; use datafusion_physical_optimizer::PhysicalOptimizerRule; +use datafusion_physical_plan::filter_pushdown::{ + FilterDescription, FilterPushdownSupport, +}; use datafusion_physical_plan::{ aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}, coalesce_batches::CoalesceBatchesExec, filter::FilterExec, repartition::RepartitionExec, + FilterPushdownResult, }; use datafusion_physical_plan::{ - displayable, filter_pushdown::FilterPushdownSupport, - metrics::ExecutionPlanMetricsSet, DisplayFormatType, ExecutionPlan, + displayable, metrics::ExecutionPlanMetricsSet, DisplayFormatType, ExecutionPlan, }; use object_store::ObjectStore; use std::sync::{Arc, OnceLock}; @@ -62,13 +65,13 @@ use std::{ /// A placeholder data source that accepts filter pushdown #[derive(Clone, Default)] struct TestSource { - support: Option, - predicate: Option, + support: bool, + predicate: Option>, statistics: Option, } impl TestSource { - fn new(support: Option) -> Self { + fn new(support: bool) -> Self { Self { support, predicate: None, @@ -148,32 +151,31 @@ impl FileSource for TestSource { fn try_pushdown_filters( &self, - filters: &[PhysicalExprRef], + fd: FilterDescription, config: &ConfigOptions, - ) -> Result { - let support = match self.support { - Some(support) => support, - None => { - if config.execution.parquet.pushdown_filters { - FilterPushdownSupport::Exact - } else { - FilterPushdownSupport::Unsupported - } + ) -> Result>> { + if self.support { + if config.execution.parquet.pushdown_filters { + return Ok(FilterPushdownSupport { + child_filters: vec![], + remaining_filters: FilterDescription { filters: vec![] }, + op: Arc::new(TestSource { + support: self.support, + predicate: Some(conjunction(fd.filters)), + statistics: self.statistics.clone(), + }), + }); } - }; - let new = Arc::new(TestSource { - support: self.support, - predicate: Some(conjunction(filters.iter().map(Arc::clone))), - statistics: self.statistics.clone(), - }); - Ok(FileSourceFilterPushdownResult::new( - new, - vec![support; filters.len()], - )) + } + Ok(FilterPushdownSupport { + child_filters: vec![], + remaining_filters: fd, + op: self, + }) } } -fn test_scan(support: Option) -> Arc { +fn test_scan(support: bool) -> Arc { let schema = schema(); let source = Arc::new(TestSource::new(support)); let base_config = FileScanConfigBuilder::new( @@ -187,7 +189,7 @@ fn test_scan(support: Option) -> Arc { #[test] fn test_pushdown_into_scan() { - let scan = test_scan(Some(FilterPushdownSupport::Exact)); + let scan = test_scan(true); let predicate = col_lit_predicate("a", "foo", schema()); let plan = Arc::new(FilterExec::try_new(predicate, scan).unwrap()); @@ -209,7 +211,7 @@ fn test_pushdown_into_scan() { /// Show that we can use config options to determine how to do pushdown. #[test] fn test_pushdown_into_scan_with_config_options() { - let scan = test_scan(None); + let scan = test_scan(false); let predicate = col_lit_predicate("a", "foo", schema()); let plan = Arc::new(FilterExec::try_new(predicate, scan).unwrap()) as _; @@ -255,7 +257,7 @@ fn test_pushdown_into_scan_with_config_options() { #[test] fn test_filter_collapse() { // filter should be pushed down into the parquet scan with two filters - let scan = test_scan(Some(FilterPushdownSupport::Exact)); + let scan = test_scan(true); let predicate1 = col_lit_predicate("a", "foo", schema()); let filter1 = Arc::new(FilterExec::try_new(predicate1, scan).unwrap()); let predicate2 = col_lit_predicate("b", "bar", schema()); @@ -278,7 +280,7 @@ fn test_filter_collapse() { #[test] fn test_filter_with_projection() { - let scan = test_scan(Some(FilterPushdownSupport::Exact)); + let scan = test_scan(true); let projection = vec![1, 0]; let projected_schema = Arc::new(schema().project(&projection).unwrap()); let predicate = col_lit_predicate("a", "foo", &projected_schema); @@ -309,7 +311,7 @@ fn test_filter_with_projection() { #[test] fn test_push_down_through_transparent_nodes() { // expect the predicate to be pushed down into the DataSource - let scan = test_scan(Some(FilterPushdownSupport::Exact)); + let scan = test_scan(true); let coalesce = Arc::new(CoalesceBatchesExec::new(scan, 1)); let predicate = col_lit_predicate("a", "foo", schema()); let filter = Arc::new(FilterExec::try_new(predicate, coalesce).unwrap()); @@ -345,7 +347,7 @@ fn test_no_pushdown_through_aggregates() { // 1. The outer filter is not pushed down into the aggregate because we haven't // implemented that yet. // 2. The inner filter **is** pushed down into the DataSource. - let scan = test_scan(Some(FilterPushdownSupport::Exact)); + let scan = test_scan(true); let filter = Arc::new( FilterExec::try_new(col_lit_predicate("a", "foo", schema()), scan.clone()) .unwrap(), diff --git a/datafusion/datasource/src/file.rs b/datafusion/datasource/src/file.rs index af0c68607edfd..f70ebfcf0525f 100644 --- a/datafusion/datasource/src/file.rs +++ b/datafusion/datasource/src/file.rs @@ -28,8 +28,10 @@ use crate::file_stream::FileOpener; use arrow::datatypes::SchemaRef; use datafusion_common::config::ConfigOptions; use datafusion_common::{Result, Statistics}; -use datafusion_physical_expr::{LexOrdering, PhysicalExprRef}; -use datafusion_physical_plan::filter_pushdown::FilterPushdownResult; +use datafusion_physical_expr::LexOrdering; +use datafusion_physical_plan::filter_pushdown::{ + FilterDescription, FilterPushdownSupport, +}; use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; use datafusion_physical_plan::DisplayFormatType; @@ -101,11 +103,11 @@ pub trait FileSource: Send + Sync { /// [`ExecutionPlan::try_pushdown_filters`]: datafusion_physical_plan::ExecutionPlan::try_pushdown_filters fn try_pushdown_filters( &self, - _filters: &[PhysicalExprRef], + fd: FilterDescription, _config: &ConfigOptions, - ) -> Result { - Ok(FileSourceFilterPushdownResult::NotPushed) + ) -> Result>> { + Ok(FilterPushdownSupport::NotSupported(fd)) } } -pub type FileSourceFilterPushdownResult = FilterPushdownResult>; +pub type FileSourceFilterPushdownResult = FilterPushdownSupport>; diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index f8235e7cff673..025ecfb30f712 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -23,6 +23,16 @@ use std::{ fmt::Result as FmtResult, marker::PhantomData, sync::Arc, }; +use crate::file_groups::FileGroup; +use crate::{ + display::FileGroupsDisplay, + file::{FileSource, FileSourceFilterPushdownResult}, + file_compression_type::FileCompressionType, + file_stream::FileStream, + source::{DataSource, DataSourceExec}, + statistics::MinMaxStatistics, + PartitionedFile, +}; use arrow::{ array::{ ArrayData, ArrayRef, BufferBuilder, DictionaryArray, RecordBatch, @@ -42,6 +52,9 @@ use datafusion_physical_expr::{ expressions::Column, EquivalenceProperties, LexOrdering, Partitioning, PhysicalExprRef, PhysicalSortExpr, }; +use datafusion_physical_plan::filter_pushdown::{ + FilterDescription, FilterPushdownSupport, +}; use datafusion_physical_plan::{ display::{display_orderings, ProjectSchemaDisplay}, metrics::ExecutionPlanMetricsSet, @@ -50,17 +63,6 @@ use datafusion_physical_plan::{ }; use log::{debug, warn}; -use crate::{ - display::FileGroupsDisplay, - file::{FileSource, FileSourceFilterPushdownResult}, - file_compression_type::FileCompressionType, - file_stream::FileStream, - source::{DataSource, DataSourceExec}, - statistics::MinMaxStatistics, - PartitionedFile, -}; -use crate::{file_groups::FileGroup, source::DataSourceFilterPushdownResult}; - /// The base configurations for a [`DataSourceExec`], the a physical plan for /// any given file format. /// @@ -589,24 +591,29 @@ impl DataSource for FileScanConfig { fn try_pushdown_filters( &self, - filters: &[PhysicalExprRef], + fd: FilterDescription, config: &ConfigOptions, - ) -> Result { - match self.file_source.try_pushdown_filters(filters, config)? { - FileSourceFilterPushdownResult::NotPushed => { - Ok(DataSourceFilterPushdownResult::NotPushed) - } - FileSourceFilterPushdownResult::Pushed { inner, support } => { - let new_self = Arc::new( + ) -> Result>> { + match self.file_source.try_pushdown_filters(fd, config)? { + FilterPushdownSupport::Supported { + child_filters, + remaining_filters, + op: file_source, + } => { + let new_data_source = Arc::new( FileScanConfigBuilder::from(self.clone()) - .with_source(inner) + .with_source(file_source) .build(), ); - Ok(DataSourceFilterPushdownResult::Pushed { - inner: new_self, - support, + Ok(FilterPushdownSupport::Supported { + child_filters, + remaining_filters, + op: new_data_source, }) } + FilterPushdownSupport::NotSupported(fd) => { + Ok(FilterPushdownSupport::NotSupported(fd)) + } } } } diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index 9df818c941a0a..781e01a53a6f0 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -26,16 +26,18 @@ use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; use datafusion_physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use datafusion_physical_plan::projection::ProjectionExec; use datafusion_physical_plan::{ - DisplayAs, DisplayFormatType, ExecutionPlan, ExecutionPlanFilterPushdownResult, - FilterPushdownResult, PlanProperties, + DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, }; use crate::file_scan_config::FileScanConfig; use datafusion_common::config::ConfigOptions; use datafusion_common::{Constraints, Result, Statistics}; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; -use datafusion_physical_expr::{EquivalenceProperties, Partitioning, PhysicalExprRef}; +use datafusion_physical_expr::{EquivalenceProperties, Partitioning}; use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_physical_plan::filter_pushdown::{ + FilterDescription, FilterPushdownSupport, +}; /// Common behaviors in Data Sources for both from Files and Memory. /// @@ -84,15 +86,13 @@ pub trait DataSource: Send + Sync + Debug { /// See [`ExecutionPlan::try_pushdown_filters`] for more details. fn try_pushdown_filters( &self, - _filters: &[PhysicalExprRef], + fd: FilterDescription, _config: &ConfigOptions, - ) -> Result { - Ok(DataSourceFilterPushdownResult::NotPushed) + ) -> Result>> { + Ok(FilterPushdownSupport::NotSupported(fd)) } } -pub type DataSourceFilterPushdownResult = FilterPushdownResult>; - /// [`ExecutionPlan`] handles different file formats like JSON, CSV, AVRO, ARROW, PARQUET /// /// `DataSourceExec` implements common functionality such as applying projections, @@ -207,24 +207,26 @@ impl ExecutionPlan for DataSourceExec { fn try_pushdown_filters( &self, - _plan: &Arc, - parent_filters: &[PhysicalExprRef], + fd: FilterDescription, config: &ConfigOptions, - ) -> Result { - match self - .data_source - .try_pushdown_filters(parent_filters, config)? - { - DataSourceFilterPushdownResult::NotPushed => { - Ok(ExecutionPlanFilterPushdownResult::NotPushed) - } - DataSourceFilterPushdownResult::Pushed { inner, support } => { - let new_self = Arc::new(DataSourceExec::new(inner)); - Ok(ExecutionPlanFilterPushdownResult::Pushed { - inner: new_self, - support, + ) -> Result>> { + let mut exec = self.clone(); + match self.data_source.try_pushdown_filters(fd, config)? { + FilterPushdownSupport::Supported { + child_filters, + remaining_filters, + op, + } => { + exec.data_source = op; + Ok(FilterPushdownSupport::Supported { + child_filters, + remaining_filters, + op: Arc::new(exec), }) } + FilterPushdownSupport::NotSupported(fd) => { + Ok(FilterPushdownSupport::NotSupported(fd)) + } } } } diff --git a/datafusion/physical-optimizer/src/filter_pushdown.rs b/datafusion/physical-optimizer/src/filter_pushdown.rs index c01f14e79c8d2..1b35f0268b85d 100644 --- a/datafusion/physical-optimizer/src/filter_pushdown.rs +++ b/datafusion/physical-optimizer/src/filter_pushdown.rs @@ -17,12 +17,16 @@ use std::sync::Arc; +use crate::PhysicalOptimizerRule; +use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{config::ConfigOptions, DataFusionError, Result}; -use datafusion_physical_plan::{ - execution_plan::ExecutionPlanFilterPushdownResult, ExecutionPlan, +use datafusion_physical_expr::conjunction; +use datafusion_physical_plan::filter::FilterExec; +use datafusion_physical_plan::filter_pushdown::{ + FilterDescription, FilterPushdownSupport, }; - -use crate::PhysicalOptimizerRule; +use datafusion_physical_plan::tree_node::PlanContext; +use datafusion_physical_plan::ExecutionPlan; /// A physical optimizer rule that pushes down filters in the execution plan. /// See [`ExecutionPlan::try_pushdown_filters`] for a detailed description of the algorithm. @@ -41,25 +45,53 @@ impl PushdownFilter { } } +pub type FilterDescriptionContext = PlanContext; + impl PhysicalOptimizerRule for PushdownFilter { fn optimize( &self, plan: Arc, config: &ConfigOptions, ) -> Result> { - match plan.try_pushdown_filters(&plan, &Vec::new(), config)? { - ExecutionPlanFilterPushdownResult::NotPushed => Ok(plan), - ExecutionPlanFilterPushdownResult::Pushed { inner, support } => { - if !support.is_empty() { - return Err( - DataFusionError::Plan( - format!("PushdownFilter: plan returned support length does not match filters length: {} != 0", support.len() - )) - ); + let context = FilterDescriptionContext::new_default(plan); + + context + .transform_down(|mut node| { + let (mut child_filters, remaining_filters, plan) = + match node.plan.try_pushdown_filters( + FilterDescription { + filters: node.data.take_filters(), + }, + config, + )? { + FilterPushdownSupport::Supported { + child_filters, + remaining_filters, + op: plan, + } => (child_filters, remaining_filters, plan), + FilterPushdownSupport::NotSupported(fd) => { + (vec![], fd, Arc::clone(&node.plan)) + } + }; + + if remaining_filters.filters.is_empty() { + node = FilterDescriptionContext::new_default(plan); + for (child, filter) in node.children.iter_mut().zip(child_filters) { + child.data = filter; + } + } else { + let mut new_child_node = FilterDescriptionContext::new_default(plan); + new_child_node.data = child_filters.swap_remove(0); + node.plan = Arc::new(FilterExec::try_new( + conjunction(remaining_filters.filters), + Arc::clone(&new_child_node.plan), + )?); + node.children = vec![new_child_node]; + node.data = FilterDescription::default(); } - Ok(inner) - } - } + Ok(Transformed::yes(node)) + }) + .map(|updated| updated.data.plan) } fn name(&self) -> &str { diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index 77dabb72aa767..b20e6c116c60c 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -30,12 +30,13 @@ use crate::{ use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; -use datafusion_common::config::ConfigOptions; use datafusion_common::Result; use datafusion_execution::TaskContext; use crate::coalesce::{BatchCoalescer, CoalescerState}; -use crate::execution_plan::{try_pushdown_filters_to_input, CardinalityEffect}; +use crate::execution_plan::CardinalityEffect; +use crate::filter_pushdown::{FilterDescription, FilterPushdownSupport}; +use datafusion_common::config::ConfigOptions; use futures::ready; use futures::stream::{Stream, StreamExt}; @@ -216,11 +217,16 @@ impl ExecutionPlan for CoalesceBatchesExec { fn try_pushdown_filters( &self, - plan: &Arc, - parent_filters: &[datafusion_physical_expr::PhysicalExprRef], - config: &ConfigOptions, - ) -> Result { - try_pushdown_filters_to_input(plan, &self.input, parent_filters, config) + fd: FilterDescription, + _config: &ConfigOptions, + ) -> Result>> { + let child_filters = vec![fd]; + let remaining_filters = FilterDescription::default(); + Ok(FilterPushdownSupport::Supported { + child_filters, + remaining_filters, + op: Arc::new(self.clone()), + }) } } diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index 50724d7564c07..8795ad878a8a2 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -16,7 +16,7 @@ // under the License. pub use crate::display::{DefaultDisplay, DisplayAs, DisplayFormatType, VerboseDisplay}; -use crate::filter_pushdown::{FilterPushdownResult, FilterPushdownSupport}; +use crate::filter_pushdown::{FilterDescription, FilterPushdownSupport}; pub use crate::metrics::Metric; pub use crate::ordering::InputOrderMode; pub use crate::stream::EmptyRecordBatchStream; @@ -46,10 +46,10 @@ use crate::stream::RecordBatchStreamAdapter; use arrow::array::{Array, RecordBatch}; use arrow::datatypes::SchemaRef; use datafusion_common::config::ConfigOptions; -use datafusion_common::{exec_err, Constraints, DataFusionError, Result}; +use datafusion_common::{exec_err, Constraints, Result}; use datafusion_common_runtime::JoinSet; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{EquivalenceProperties, LexOrdering, PhysicalExprRef}; +use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; use datafusion_physical_expr_common::sort_expr::LexRequirement; use futures::stream::{StreamExt, TryStreamExt}; @@ -469,381 +469,12 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { Ok(None) } - /// A physical optimizer rule that pushes down filters in the execution plan. - /// For example, consider the following plan: - /// - /// ```text - /// ┌──────────────────────┐ - /// │ CoalesceBatchesExec │ - /// └──────────────────────┘ - /// │ - /// ▼ - /// ┌──────────────────────┐ - /// │ FilterExec │ - /// │ filters = [ id=1] │ - /// └──────────────────────┘ - /// │ - /// ▼ - /// ┌──────────────────────┐ - /// │ DataSourceExec │ - /// │ projection = * │ - /// └──────────────────────┘ - /// ``` - /// - /// Our goal is to move the `id = 1` filter from the [`FilterExec`] node to the `DataSourceExec` node. - /// If this filter is selective it can avoid massive amounts of data being read from the source (the projection is `*` so all matching columns are read). - /// In this simple case we: - /// 1. Enter the recursion with no filters. - /// 2. We find the [`FilterExec`] node and call [`ExecutionPlan::try_pushdown_filters`] on it. - /// 3. The [`FilterExec`] node tries to push it's filters + the filters from the parent nodes (in this case empty) - /// down into it's input, which is the `DataSourceExec` node. - /// 4. The `DataSourceExec` node accepts the filter and returns a [`FilterPushdownResult`] with a new copy of itself - /// and [`FilterPushdownSupport::Exact`] to indicate that the filter was pushed down and the caller no longer - /// needs to handle it. - /// 5. The [`FilterExec`] seeing that all filters were pushed down returns a [`FilterPushdownResult`] that directly - /// returns the new `DataSourceExec` node, effectively removing the [`FilterExec`] node from the plan. - /// - /// The new plan looks like: - /// - /// ```text - /// ┌──────────────────────┐ - /// │ CoalesceBatchesExec │ - /// └──────────────────────┘ - /// │ - /// ▼ - /// ┌──────────────────────┐ - /// │ DataSourceExec │ - // │ projection = * │ - // │ filters = [ id=1] │ - /// └──────────────────────┘ - /// ``` - /// - /// Let's consider a more complex example involving a [`ProjectionExec`] node in betweeen the [`FilterExec`] and `DataSourceExec` nodes that creates a new column that the filter depends on. - /// - /// ```text - // ┌──────────────────────┐ - // │ CoalesceBatchesExec │ - // └──────────────────────┘ - // │ - // ▼ - // ┌──────────────────────┐ - // │ FilterExec │ - // │ filters = │ - // │ [cost>50,id=1] │ - // └──────────────────────┘ - // │ - // ▼ - // ┌──────────────────────┐ - // │ ProjectionExec │ - // │ cost = price * 1.2 │ - // └──────────────────────┘ - // │ - // ▼ - // ┌──────────────────────┐ - // │ DataSourceExec │ - // │ projection = * │ - // └──────────────────────┘ - /// ``` - /// - /// We want to push down the filters [id=1] to the `DataSourceExec` node, but can't push down `cost>50` because it requires the [`ProjectionExec`] node to be executed first. - /// A simple thing to do would be to split up the filter into two separate filters and push down the first one: - /// - /// ```text - // ┌──────────────────────┐ - // │ CoalesceBatchesExec │ - // └──────────────────────┘ - // │ - // ▼ - // ┌──────────────────────┐ - // │ FilterExec │ - // │ filters = │ - // │ [cost>50] │ - // └──────────────────────┘ - // │ - // ▼ - // ┌──────────────────────┐ - // │ ProjectionExec │ - // │ cost = price * 1.2 │ - // └──────────────────────┘ - // │ - // ▼ - // ┌──────────────────────┐ - // │ DataSourceExec │ - // │ projection = * │ - // │ filters = [ id=1] │ - // └──────────────────────┘ - /// ``` - /// - /// We can actually however do better by pushing down `price * 1.2 > 50` instead of `cost > 50`: - /// - /// ```text - /// ┌──────────────────────┐ - /// │ CoalesceBatchesExec │ - /// └──────────────────────┘ - /// │ - /// ▼ - /// ┌──────────────────────┐ - /// │ ProjectionExec │ - /// │ cost = price * 1.2 │ - /// └──────────────────────┘ - /// │ - /// ▼ - /// ┌──────────────────────┐ - /// │ DataSourceExec │ - /// │ projection = * │ - /// │ filters = [id=1, │ - /// │ price * 1.2 > 50] │ - /// └──────────────────────┘ - /// ``` - /// - /// There are also cases where we may be able to push down filters within a subtree but not the entire tree. - /// A good exmaple of this is aggreagation nodes: - /// - /// ```text - /// ┌──────────────────────┐ - /// │ ProjectionExec │ - /// │ projection = * │ - /// └──────────────────────┘ - /// │ - /// ▼ - /// ┌──────────────────────┐ - /// │ FilterExec │ - /// │ filters = [sum > 10] │ - /// └──────────────────────┘ - /// │ - /// ▼ - /// ┌───────────────────────┐ - /// │ AggregateExec │ - /// │ group by = [id] │ - /// │ aggregate = │ - /// │ [sum(price)] │ - /// └───────────────────────┘ - /// │ - /// ▼ - /// ┌──────────────────────┐ - /// │ FilterExec │ - /// │ filters = [id=1] │ - /// └──────────────────────┘ - /// │ - /// ▼ - /// ┌──────────────────────┐ - /// │ DataSourceExec │ - /// │ projection = * │ - /// └──────────────────────┘ - /// ``` - /// - /// The transformation here is to push down the `id=1` filter to the `DataSourceExec` node: - /// - /// ```text - /// ┌──────────────────────┐ - /// │ ProjectionExec │ - /// │ projection = * │ - /// └──────────────────────┘ - /// │ - /// ▼ - /// ┌──────────────────────┐ - /// │ FilterExec │ - /// │ filters = [sum > 10] │ - /// └──────────────────────┘ - /// │ - /// ▼ - /// ┌───────────────────────┐ - /// │ AggregateExec │ - /// │ group by = [id] │ - /// │ aggregate = │ - /// │ [sum(price)] │ - /// └───────────────────────┘ - /// │ - /// ▼ - /// ┌──────────────────────┐ - /// │ DataSourceExec │ - /// │ projection = * │ - /// │ filters = [id=1] │ - /// └──────────────────────┘ - /// ``` - /// - /// The point here is that: - /// 1. We cannot push down `sum > 10` through the `AggregateExec` node into the `DataSourceExec` node. - /// Any filters above the `AggregateExec` node are not pushed down. - /// This is determined by calling [`ExecutionPlan::try_pushdown_filters`] on the [`AggregateExec`] node. - /// 2. We need to keep recursing into the tree so that we can discover the other [`FilterExec`] node and push down the `id=1` filter. - /// - /// It is also possible to push down filters through joins and from joins. - /// For example, a hash join where we build a hash table of the left side and probe the right side - /// (ignoring why we would choose this order, typically it depends on the size of each table, etc.). - /// - /// ```text - /// ┌─────────────────────┐ - /// │ FilterExec │ - /// │ filters = │ - /// │ [d.size > 100] │ - /// └─────────────────────┘ - /// │ - /// │ - /// ┌──────────▼──────────┐ - /// │ │ - /// │ HashJoinExec │ - /// │ [u.dept@hash(d.id)] │ - /// │ │ - /// └─────────────────────┘ - /// │ - /// ┌────────────┴────────────┐ - /// ┌──────────▼──────────┐ ┌──────────▼──────────┐ - /// │ DataSourceExec │ │ DataSourceExec │ - /// │ alias [users as u] │ │ alias [dept as d] │ - /// │ │ │ │ - /// └─────────────────────┘ └─────────────────────┘ - /// ``` - /// - /// There are two pushdowns we can do here: - /// 1. Push down the `d.size > 100` filter through the `HashJoinExec` node to the `DataSourceExec` node for the `departments` table. - /// 2. Push down the hash table state from the `HashJoinExec` node to the `DataSourceExec` node to avoid reading - /// rows from teh `users` table that will be eliminated by the join. - /// This can be done via a bloom filter or similar. - /// - /// ```text - /// ┌─────────────────────┐ - /// │ │ - /// │ HashJoinExec │ - /// │ [u.dept@hash(d.id)] │ - /// │ │ - /// └─────────────────────┘ - /// │ - /// ┌────────────┴────────────┐ - /// ┌──────────▼──────────┐ ┌──────────▼──────────┐ - /// │ DataSourceExec │ │ DataSourceExec │ - /// │ alias [users as u] │ │ alias [dept as d] │ - /// │ filters = │ │ filters = │ - /// │ [depg@hash(d.id)] │ │ [ d.size > 100] │ - /// └─────────────────────┘ └─────────────────────┘ - /// ``` - /// - /// You may notice in this case that the filter is *dynamic*: the hash table is built - /// _after_ the `departments` table is read and at runtime. - /// We don't have a concrete `InList` filter or similar to push down at optimization time. - /// These sorts of dynamic filters are handled by building a specialized - /// [`PhysicalExpr`] that can be evaluated at runtime - /// and internally maintains a reference to the hash table or other state. - /// To make working with these sorts of dynamic filters more tractable we have the method `PhysicalExpr::snapshot` - /// (TODO: add reference after is merged) - /// which attempts to simplify a dynamic filter into a "basic" non-dynamic filter. - /// For a join this could mean converting it to an `InList` filter or a min/max filter for example. - /// See `datafusion/physical-plan/src/dynamic_filters.rs` for more details. - /// - /// Another form of dyanmic filter is pushing down the state of a `TopK` operator for queries like - /// `SELECT * FROM t ORDER BY id LIMIT 10`: - /// - /// ```text - /// ┌──────────────────────┐ - /// │ TopK │ - /// │ limit = 10 │ - /// │ order by = [id] │ - /// └──────────────────────┘ - /// │ - /// ▼ - /// ┌──────────────────────┐ - /// │ DataSourceExec │ - /// │ projection = * │ - /// └──────────────────────┘ - /// ``` - /// - /// We can avoid large amounts of data processing by transforming this into: - /// - /// ```text - /// ┌──────────────────────┐ - /// │ TopK │ - /// │ limit = 10 │ - /// │ order by = [id] │ - /// └──────────────────────┘ - /// │ - /// ▼ - /// ┌──────────────────────┐ - /// │ DataSourceExec │ - /// │ projection = * │ - /// │ filters = │ - /// │ [id < @ TopKHeap] │ - /// └──────────────────────┘ - /// ``` - /// - /// Now as we fill our `TopK` heap we can push down the state of the heap to the `DataSourceExec` node - /// to avoid reading files / row groups / pages / rows that could not possibly be in the top 10. - /// This is implemented in datafusion/physical-plan/src/sorts/sort_filters.rs. - /// - /// [`FilterExec`]: crate::filter::FilterExec - /// [`ProjectionExec`]: crate::projection::ProjectionExec - /// [`AggregateExec`]: crate::aggregates::AggregateExec fn try_pushdown_filters( &self, - plan: &Arc, - parent_filters: &[PhysicalExprRef], - config: &ConfigOptions, - ) -> Result { - // By default assume that: - // * Parent filters can't be passed onto children. - // * We have no filters to contribute. - // But we still want to recurse into our children in case a subtree has pushdowns within - // it and thus we need to replace our children with the new plans. - let mut new_children = Vec::with_capacity(self.children().len()); - let mut pushed = false; - for child in self.children() { - match child.try_pushdown_filters(child, &Vec::new(), config)? { - ExecutionPlanFilterPushdownResult::NotPushed => { - // No pushdown possible, keep this child as is - new_children.push(Arc::clone(child)); - } - ExecutionPlanFilterPushdownResult::Pushed { inner, support } => { - // We have a child that has pushed down some filters - new_children.push(inner); - pushed = true; - // Support should be empty, we didn't pass any filters - if !support.is_empty() { - return Err(DataFusionError::Internal( - "Child plan did not have any filters pushed down".to_string(), - )); - } - } - } - } - if pushed { - let new_inner = - with_new_children_if_necessary(Arc::clone(plan), new_children)?; - Ok(ExecutionPlanFilterPushdownResult::Pushed { - inner: new_inner, - support: vec![FilterPushdownSupport::Unsupported; parent_filters.len()], - }) - } else { - Ok(ExecutionPlanFilterPushdownResult::NotPushed) - } - } -} - -pub type ExecutionPlanFilterPushdownResult = FilterPushdownResult>; - -/// A default implementation of [`ExecutionPlan::try_pushdown_filters`] that -/// pushes down filters transparently to the input of the plan for plans that: -/// * Have a single input / child node. -/// * Support transparent filter pushdown (do not modify the cardinality or schema of the data). -/// * Do not have any filters of their own. -pub fn try_pushdown_filters_to_input( - plan: &Arc, - input: &Arc, - parent_filters: &[PhysicalExprRef], - config: &ConfigOptions, -) -> Result { - match input.try_pushdown_filters(input, parent_filters, config)? { - ExecutionPlanFilterPushdownResult::NotPushed => { - // No pushdown possible, keep this child as is - Ok(ExecutionPlanFilterPushdownResult::NotPushed) - } - ExecutionPlanFilterPushdownResult::Pushed { inner, support } => { - // We have a child that has pushed down some filters - let new_inner = - with_new_children_if_necessary(Arc::clone(plan), vec![inner])?; - Ok(ExecutionPlanFilterPushdownResult::Pushed { - inner: new_inner, - support, - }) - } + fd: FilterDescription, + _config: &ConfigOptions, + ) -> Result>> { + Ok(FilterPushdownSupport::NotSupported(fd)) } } diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index e542b383bbd2b..238e065a37a98 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -26,12 +26,11 @@ use super::{ }; use crate::common::can_project; use crate::execution_plan::CardinalityEffect; -use crate::filter_pushdown::FilterPushdownSupport; +use crate::filter_pushdown::{FilterDescription, FilterPushdownSupport}; use crate::projection::{ make_with_child, try_embed_projection, update_expr, EmbeddedProjection, ProjectionExec, }; -use crate::ExecutionPlanFilterPushdownResult; use crate::{ metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}, DisplayFormatType, ExecutionPlan, @@ -51,7 +50,7 @@ use datafusion_expr::Operator; use datafusion_physical_expr::equivalence::ProjectionMapping; use datafusion_physical_expr::expressions::BinaryExpr; use datafusion_physical_expr::intervals::utils::check_support; -use datafusion_physical_expr::utils::{collect_columns, reassign_predicate_columns}; +use datafusion_physical_expr::utils::collect_columns; use datafusion_physical_expr::{ analyze, conjunction, split_conjunction, AcrossPartitions, AnalysisContext, ConstExpr, ExprBoundaries, PhysicalExpr, PhysicalExprRef, @@ -439,77 +438,16 @@ impl ExecutionPlan for FilterExec { fn try_pushdown_filters( &self, - _plan: &Arc, - parent_filters: &[PhysicalExprRef], - config: &ConfigOptions, - ) -> Result { - let mut all_filters = parent_filters.to_vec(); - all_filters.push(Arc::clone(&self.predicate)); - let all_filters = if self.projection.is_some() { - let input_schema = self.input.schema(); - all_filters - .into_iter() - .map(|f| reassign_predicate_columns(f, &input_schema, false)) - .collect::>>()? - } else { - all_filters - }; - let (new_predicate, new_input) = - match self - .input - .try_pushdown_filters(&self.input, &all_filters, config)? - { - ExecutionPlanFilterPushdownResult::NotPushed => { - if parent_filters.is_empty() { - return Ok(ExecutionPlanFilterPushdownResult::NotPushed); - } - (conjunction(all_filters), Arc::clone(&self.input)) - } - ExecutionPlanFilterPushdownResult::Pushed { inner, support } => { - // Split out the filters that the child plan handled and the ones it did not - let unhandled_filters = all_filters - .into_iter() - .zip(support) - .filter_map(|(f, s)| { - if matches!(s, FilterPushdownSupport::Exact) { - None - } else { - Some(f) - } - }) - .collect::>(); - // If there are no unhandled filters and we have no projection, return the inner plan - if unhandled_filters.is_empty() && self.projection.is_none() { - return Ok(ExecutionPlanFilterPushdownResult::Pushed { - inner, - support: vec![ - FilterPushdownSupport::Exact; - parent_filters.len() - ], - }); - } - let new_predicate = conjunction(unhandled_filters); - (new_predicate, inner) - } - }; - - let cache = Self::compute_properties( - &self.input, - &new_predicate, - self.default_selectivity, - self.projection.as_ref(), - )?; - let new_self = Self { - predicate: new_predicate, - input: new_input, - metrics: self.metrics.clone(), - default_selectivity: self.default_selectivity, - cache, - projection: self.projection.clone(), - }; - Ok(ExecutionPlanFilterPushdownResult::Pushed { - inner: Arc::new(new_self), - support: vec![FilterPushdownSupport::Exact; parent_filters.len()], + mut fd: FilterDescription, + _config: &ConfigOptions, + ) -> Result>> { + fd.filters.push(Arc::clone(self.predicate())); + let child_filters = vec![fd]; + let remaining_filters = FilterDescription { filters: vec![] }; + Ok(FilterPushdownSupport::Supported { + child_filters, + remaining_filters, + op: Arc::clone(&self.input), }) } } diff --git a/datafusion/physical-plan/src/filter_pushdown.rs b/datafusion/physical-plan/src/filter_pushdown.rs index d2c60e5d2d9fd..26ac480c6775b 100644 --- a/datafusion/physical-plan/src/filter_pushdown.rs +++ b/datafusion/physical-plan/src/filter_pushdown.rs @@ -15,49 +15,37 @@ // specific language governing permissions and limitations // under the License. -/// Result of trying to push down fitlers to a child plan. -/// This is used by [`FilterPushdownResult`] to indicate whether the filter was -/// "absorbed" by the child ([`FilterPushdownSupport::Exact`]) or not -/// ([`FilterPushdownSupport::Unsupported`]). -/// If the filter was not absorbed, the parent plan must apply the filter -/// itself, or return to the caller that it was not pushed down. -/// If the filter was absorbed, the parent plan can drop the filter or -/// tell the caller that it was pushed down by forwarding on the [`FilterPushdownSupport::Exact`] -/// information. -#[derive(Debug, Clone, Copy)] -pub enum FilterPushdownSupport { - /// Filter may not have been pushed down to the child plan, or the child plan - /// can only partially apply the filter but may have false positives (but not false negatives). - /// In this case the parent **must** behave as if the filter was not pushed down - /// and must apply the filter itself. - Unsupported, - /// Filter was pushed down to the child plan and the child plan promises that - /// it will apply the filter correctly with no false positives or false negatives. - /// The parent can safely drop the filter. - Exact, -} +use std::sync::Arc; -/// The combined result of a filter pushdown operation. -/// This includes: -/// * The inner plan that was produced by the pushdown operation. -/// * The support for each filter that was pushed down. -pub enum FilterPushdownResult { - /// No pushdown was possible, keep this node as is in the tree. - NotPushed, - /// Pushed some or all filters into this node. - /// The caller should replace the node in the tree with the new one provided - /// and should transmit to parents the support for each filter. - Pushed { - /// The inner node that was produced by the pushdown operation. - inner: T, - /// The support for each filter that was pushed down. - support: Vec, - }, +use crate::ExecutionPlan; + +use datafusion_common::Result; +use datafusion_physical_expr_common::physical_expr::PhysicalExpr; + +#[derive(Default)] +pub struct FilterDescription { + /// Expressions coming from the parent nodes + pub filters: Vec>, } -impl FilterPushdownResult { - /// Create a new [`FilterPushdownResult`]. - pub fn new(inner: T, support: Vec) -> Self { - Self::Pushed { inner, support } +impl FilterDescription { + /// Takes the filters out of the struct, leaving an empty vector in its place. + pub fn take_filters(&mut self) -> Vec> { + std::mem::take(&mut self.filters) } } + +pub enum FilterPushdownSupport { + Supported { + // Filter predicates which can be pushed down through the operator. + // NOTE that these are not placed into any operator. + child_filters: Vec, + // Filters which cannot be pushed down through the operator. + // NOTE that caller of try_pushdown_filters() should handle these remanining predicates, + // possibly introducing a FilterExec on top of this operator. + remaining_filters: FilterDescription, + // Possibly updated new operator + op: T, + }, + NotSupported(FilterDescription), +} diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 64e3d6f39a18f..32c6ece6456e8 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -43,10 +43,8 @@ pub use crate::display::{DefaultDisplay, DisplayAs, DisplayFormatType, VerboseDi pub use crate::execution_plan::{ collect, collect_partitioned, displayable, execute_input_stream, execute_stream, execute_stream_partitioned, get_plan_string, with_new_children_if_necessary, - ExecutionPlan, ExecutionPlanFilterPushdownResult, ExecutionPlanProperties, - PlanProperties, + ExecutionPlan, ExecutionPlanProperties, PlanProperties, }; -pub use crate::filter_pushdown::FilterPushdownResult; pub use crate::metrics::Metric; pub use crate::ordering::InputOrderMode; pub use crate::stream::EmptyRecordBatchStream; diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 53d53900af2d6..50aa7a4605452 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -29,7 +29,7 @@ use super::metrics::{self, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; use super::{ DisplayAs, ExecutionPlanProperties, RecordBatchStream, SendableRecordBatchStream, }; -use crate::execution_plan::{try_pushdown_filters_to_input, CardinalityEffect}; +use crate::execution_plan::CardinalityEffect; use crate::hash_utils::create_hashes; use crate::metrics::BaselineMetrics; use crate::projection::{all_columns, make_with_child, update_expr, ProjectionExec}; @@ -53,6 +53,7 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::{EquivalenceProperties, PhysicalExpr}; use datafusion_physical_expr_common::sort_expr::LexOrdering; +use crate::filter_pushdown::{FilterDescription, FilterPushdownSupport}; use futures::stream::Stream; use futures::{FutureExt, StreamExt, TryStreamExt}; use log::trace; @@ -727,11 +728,16 @@ impl ExecutionPlan for RepartitionExec { fn try_pushdown_filters( &self, - plan: &Arc, - parent_filters: &[datafusion_physical_expr::PhysicalExprRef], - config: &ConfigOptions, - ) -> Result { - try_pushdown_filters_to_input(plan, &self.input, parent_filters, config) + fd: FilterDescription, + _config: &ConfigOptions, + ) -> Result>> { + let child_filters = vec![fd]; + let remaining_filters = FilterDescription::default(); + Ok(FilterPushdownSupport::Supported { + child_filters, + remaining_filters, + op: Arc::new(self.clone()), + }) } } From cda6e8d6b74e860e9978abf988c00d5a00d1ac07 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Mon, 14 Apr 2025 08:36:04 -0500 Subject: [PATCH 23/40] re-add docs --- .../physical_optimizer/filter_pushdown.rs | 49 ++- datafusion/datasource/src/file.rs | 2 - datafusion/datasource/src/file_scan_config.rs | 4 +- datafusion/datasource/src/source.rs | 8 +- .../physical-optimizer/src/filter_pushdown.rs | 348 +++++++++++++++++- .../physical-plan/src/coalesce_batches.rs | 4 +- .../physical-plan/src/execution_plan.rs | 13 +- datafusion/physical-plan/src/filter.rs | 17 +- .../physical-plan/src/filter_pushdown.rs | 3 - .../physical-plan/src/repartition/mod.rs | 4 +- 10 files changed, 415 insertions(+), 37 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs index ce1eed2a13588..12bbb480e9161 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs @@ -30,14 +30,12 @@ use datafusion_common::{internal_err, Result}; use datafusion_datasource::file_scan_config::FileScanConfigBuilder; use datafusion_datasource::source::DataSourceExec; use datafusion_datasource::{ - file::{FileSource, FileSourceFilterPushdownResult}, - file_scan_config::FileScanConfig, - file_stream::FileOpener, + file::FileSource, file_scan_config::FileScanConfig, file_stream::FileOpener, }; use datafusion_expr::test::function_stub::count_udaf; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::{ - aggregate::AggregateExprBuilder, conjunction, Partitioning, PhysicalExprRef, + aggregate::AggregateExprBuilder, conjunction, Partitioning, }; use datafusion_physical_expr_common::physical_expr::fmt_sql; use datafusion_physical_optimizer::filter_pushdown::PushdownFilter; @@ -50,7 +48,6 @@ use datafusion_physical_plan::{ coalesce_batches::CoalesceBatchesExec, filter::FilterExec, repartition::RepartitionExec, - FilterPushdownResult, }; use datafusion_physical_plan::{ displayable, metrics::ExecutionPlanMetricsSet, DisplayFormatType, ExecutionPlan, @@ -156,7 +153,7 @@ impl FileSource for TestSource { ) -> Result>> { if self.support { if config.execution.parquet.pushdown_filters { - return Ok(FilterPushdownSupport { + return Ok(FilterPushdownSupport::Supported { child_filters: vec![], remaining_filters: FilterDescription { filters: vec![] }, op: Arc::new(TestSource { @@ -167,10 +164,10 @@ impl FileSource for TestSource { }); } } - Ok(FilterPushdownSupport { + Ok(FilterPushdownSupport::Supported { child_filters: vec![], remaining_filters: fd, - op: self, + op: Arc::new(self.clone()), }) } } @@ -285,7 +282,7 @@ fn test_filter_with_projection() { let projected_schema = Arc::new(schema().project(&projection).unwrap()); let predicate = col_lit_predicate("a", "foo", &projected_schema); let plan = Arc::new( - FilterExec::try_new(predicate, scan) + FilterExec::try_new(predicate, Arc::clone(&scan)) .unwrap() .with_projection(Some(projection)) .unwrap(), @@ -295,16 +292,44 @@ fn test_filter_with_projection() { insta::assert_snapshot!( OptimizationTest::new(plan, PushdownFilter{}), - @r" + @r#" OptimizationTest: input: - FilterExec: a@1 = foo, projection=[b@1, a@0] - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test + output: + Err: Internal error: Schema mismatch: + + Before: + Schema { fields: [Field { name: "b", data_type: Utf8, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }, Field { name: "a", data_type: Utf8, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }], metadata: {} } + + After: + Schema { fields: [Field { name: "a", data_type: Utf8, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }, Field { name: "b", data_type: Utf8, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }, Field { name: "c", data_type: Float64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }], metadata: {} }. + This was likely caused by a bug in DataFusion's code and we would welcome that you file an bug report in our issue tracker + "#, + ); + + // add a test where the filter is on a column that isn't included in the output + let projection = vec![1]; + let predicate = col_lit_predicate("a", "foo", &schema()); + let plan = Arc::new( + FilterExec::try_new(predicate, scan) + .unwrap() + .with_projection(Some(projection)) + .unwrap(), + ); + insta::assert_snapshot!( + OptimizationTest::new(plan, PushdownFilter{}), + @r" + OptimizationTest: + input: + - FilterExec: a@0 = foo, projection=[b@1] + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test output: Ok: - - FilterExec: true, projection=[b@1, a@0] + - FilterExec: true, projection=[b@1] - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, predicate=a@0 = foo - ", + " ); } diff --git a/datafusion/datasource/src/file.rs b/datafusion/datasource/src/file.rs index f70ebfcf0525f..8508bbb9dc7b9 100644 --- a/datafusion/datasource/src/file.rs +++ b/datafusion/datasource/src/file.rs @@ -109,5 +109,3 @@ pub trait FileSource: Send + Sync { Ok(FilterPushdownSupport::NotSupported(fd)) } } - -pub type FileSourceFilterPushdownResult = FilterPushdownSupport>; diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index 025ecfb30f712..667db9f164e64 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -26,7 +26,7 @@ use std::{ use crate::file_groups::FileGroup; use crate::{ display::FileGroupsDisplay, - file::{FileSource, FileSourceFilterPushdownResult}, + file::FileSource, file_compression_type::FileCompressionType, file_stream::FileStream, source::{DataSource, DataSourceExec}, @@ -50,7 +50,7 @@ use datafusion_execution::{ }; use datafusion_physical_expr::{ expressions::Column, EquivalenceProperties, LexOrdering, Partitioning, - PhysicalExprRef, PhysicalSortExpr, + PhysicalSortExpr, }; use datafusion_physical_plan::filter_pushdown::{ FilterDescription, FilterPushdownSupport, diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index 781e01a53a6f0..2a087aaff9cc3 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -206,12 +206,12 @@ impl ExecutionPlan for DataSourceExec { } fn try_pushdown_filters( - &self, + self: Arc, fd: FilterDescription, config: &ConfigOptions, ) -> Result>> { - let mut exec = self.clone(); - match self.data_source.try_pushdown_filters(fd, config)? { + let mut exec = Arc::unwrap_or_clone(self); + match exec.data_source.try_pushdown_filters(fd, config)? { FilterPushdownSupport::Supported { child_filters, remaining_filters, @@ -221,7 +221,7 @@ impl ExecutionPlan for DataSourceExec { Ok(FilterPushdownSupport::Supported { child_filters, remaining_filters, - op: Arc::new(exec), + op: Arc::new(exec) as Arc, }) } FilterPushdownSupport::NotSupported(fd) => { diff --git a/datafusion/physical-optimizer/src/filter_pushdown.rs b/datafusion/physical-optimizer/src/filter_pushdown.rs index 1b35f0268b85d..c076b397cfd41 100644 --- a/datafusion/physical-optimizer/src/filter_pushdown.rs +++ b/datafusion/physical-optimizer/src/filter_pushdown.rs @@ -19,7 +19,7 @@ use std::sync::Arc; use crate::PhysicalOptimizerRule; use datafusion_common::tree_node::{Transformed, TreeNode}; -use datafusion_common::{config::ConfigOptions, DataFusionError, Result}; +use datafusion_common::{config::ConfigOptions, Result}; use datafusion_physical_expr::conjunction; use datafusion_physical_plan::filter::FilterExec; use datafusion_physical_plan::filter_pushdown::{ @@ -28,8 +28,348 @@ use datafusion_physical_plan::filter_pushdown::{ use datafusion_physical_plan::tree_node::PlanContext; use datafusion_physical_plan::ExecutionPlan; -/// A physical optimizer rule that pushes down filters in the execution plan. -/// See [`ExecutionPlan::try_pushdown_filters`] for a detailed description of the algorithm. +/// Attempts to recursively push given filters from the top of the tree into leafs. +/// +/// # Default Implementation +/// +/// The default implementation in [`ExecutionPlan::try_pushdown_filters`] is a no-op +/// that assumes that: +/// +/// * Parent filters can't be passed onto children. +/// * This node has no filters to contribute. +/// +/// # Example: Push filter into a `DataSourceExec` +/// +/// For example, consider the following plan: +/// +/// ```text +/// ┌──────────────────────┐ +/// │ CoalesceBatchesExec │ +/// └──────────────────────┘ +/// │ +/// ▼ +/// ┌──────────────────────┐ +/// │ FilterExec │ +/// │ filters = [ id=1] │ +/// └──────────────────────┘ +/// │ +/// ▼ +/// ┌──────────────────────┐ +/// │ DataSourceExec │ +/// │ projection = * │ +/// └──────────────────────┘ +/// ``` +/// +/// Our goal is to move the `id = 1` filter from the [`FilterExec`] node to the `DataSourceExec` node. +/// +/// If this filter is selective pushing it into the scan can avoid massive +/// amounts of data being read from the source (the projection is `*` so all +/// matching columns are read). +/// +/// In this simple case we: +/// 1. Enter the recursion with no filters. +/// 2. We find the [`FilterExec`] node and call [`ExecutionPlan::try_pushdown_filters`] on it. +/// 3. The [`FilterExec`] node tries to push it's filters + the filters from the parent nodes (in this case empty) +/// down into it's input, which is the `DataSourceExec` node. +/// 4. The `DataSourceExec` node accepts the filter and returns a [`FilterPushdownResult`] with a new copy of itself +/// and [`FilterPushdown::Exact`] to indicate that the filter was pushed down and the caller no longer +/// needs to handle it. +/// 5. The [`FilterExec`] seeing that all filters were pushed down returns a [`FilterPushdownResult`] that directly +/// returns the new `DataSourceExec` node, effectively removing the [`FilterExec`] node from the plan. +/// +/// The new plan looks like: +/// +/// ```text +/// ┌──────────────────────┐ +/// │ CoalesceBatchesExec │ +/// └──────────────────────┘ +/// │ +/// ▼ +/// ┌──────────────────────┐ +/// │ DataSourceExec │ +/// │ projection = * │ +/// │ filters = [ id=1] │ +/// └──────────────────────┘ +/// ``` +/// +/// # Example: Push filters with `ProjectionExec` +/// +/// Let's consider a more complex example involving a [`ProjectionExec`] +/// node in between the [`FilterExec`] and `DataSourceExec` nodes that +/// creates a new column that the filter depends on. +/// +/// ```text +/// ┌──────────────────────┐ +/// │ CoalesceBatchesExec │ +/// └──────────────────────┘ +/// │ +/// ▼ +/// ┌──────────────────────┐ +/// │ FilterExec │ +/// │ filters = │ +/// │ [cost>50,id=1] │ +/// └──────────────────────┘ +/// │ +/// ▼ +/// ┌──────────────────────┐ +/// │ ProjectionExec │ +/// │ cost = price * 1.2 │ +/// └──────────────────────┘ +/// │ +/// ▼ +/// ┌──────────────────────┐ +/// │ DataSourceExec │ +/// │ projection = * │ +/// └──────────────────────┘ +/// ``` +/// +/// We want to push down the filters `[id=1]` to the `DataSourceExec` node, +/// but can't push down `cost>50` because it requires the [`ProjectionExec`] +/// node to be executed first. A simple thing to do would be to split up the +/// filter into two separate filters and push down the first one: +/// +/// ```text +/// ┌──────────────────────┐ +/// │ CoalesceBatchesExec │ +/// └──────────────────────┘ +/// │ +/// ▼ +/// ┌──────────────────────┐ +/// │ FilterExec │ +/// │ filters = │ +/// │ [cost>50] │ +/// └──────────────────────┘ +/// │ +/// ▼ +/// ┌──────────────────────┐ +/// │ ProjectionExec │ +/// │ cost = price * 1.2 │ +/// └──────────────────────┘ +/// │ +/// ▼ +/// ┌──────────────────────┐ +/// │ DataSourceExec │ +/// │ projection = * │ +/// │ filters = [ id=1] │ +/// └──────────────────────┘ +/// ``` +/// +/// We can actually however do better by pushing down `price * 1.2 > 50` +/// instead of `cost > 50`: +/// +/// ```text +/// ┌──────────────────────┐ +/// │ CoalesceBatchesExec │ +/// └──────────────────────┘ +/// │ +/// ▼ +/// ┌──────────────────────┐ +/// │ ProjectionExec │ +/// │ cost = price * 1.2 │ +/// └──────────────────────┘ +/// │ +/// ▼ +/// ┌──────────────────────┐ +/// │ DataSourceExec │ +/// │ projection = * │ +/// │ filters = [id=1, │ +/// │ price * 1.2 > 50] │ +/// └──────────────────────┘ +/// ``` +/// +/// # Example: Push filters within a subtree +/// +/// There are also cases where we may be able to push down filters within a +/// subtree but not the entire tree. A good example of this is aggregation +/// nodes: +/// +/// ```text +/// ┌──────────────────────┐ +/// │ ProjectionExec │ +/// │ projection = * │ +/// └──────────────────────┘ +/// │ +/// ▼ +/// ┌──────────────────────┐ +/// │ FilterExec │ +/// │ filters = [sum > 10] │ +/// └──────────────────────┘ +/// │ +/// ▼ +/// ┌───────────────────────┐ +/// │ AggregateExec │ +/// │ group by = [id] │ +/// │ aggregate = │ +/// │ [sum(price)] │ +/// └───────────────────────┘ +/// │ +/// ▼ +/// ┌──────────────────────┐ +/// │ FilterExec │ +/// │ filters = [id=1] │ +/// └──────────────────────┘ +/// │ +/// ▼ +/// ┌──────────────────────┐ +/// │ DataSourceExec │ +/// │ projection = * │ +/// └──────────────────────┘ +/// ``` +/// +/// The transformation here is to push down the `id=1` filter to the +/// `DataSourceExec` node: +/// +/// ```text +/// ┌──────────────────────┐ +/// │ ProjectionExec │ +/// │ projection = * │ +/// └──────────────────────┘ +/// │ +/// ▼ +/// ┌──────────────────────┐ +/// │ FilterExec │ +/// │ filters = [sum > 10] │ +/// └──────────────────────┘ +/// │ +/// ▼ +/// ┌───────────────────────┐ +/// │ AggregateExec │ +/// │ group by = [id] │ +/// │ aggregate = │ +/// │ [sum(price)] │ +/// └───────────────────────┘ +/// │ +/// ▼ +/// ┌──────────────────────┐ +/// │ DataSourceExec │ +/// │ projection = * │ +/// │ filters = [id=1] │ +/// └──────────────────────┘ +/// ``` +/// +/// The point here is that: +/// 1. We cannot push down `sum > 10` through the `AggregateExec` node into the `DataSourceExec` node. +/// Any filters above the `AggregateExec` node are not pushed down. +/// This is determined by calling [`ExecutionPlan::try_pushdown_filters`] on the [`AggregateExec`] node. +/// 2. We need to keep recursing into the tree so that we can discover the other [`FilterExec`] node and push +/// down the `id=1` filter. +/// +/// # Example: Push filters through Joins +/// +/// It is also possible to push down filters through joins and filters that +/// originate from joins. For example, a hash join where we build a hash +/// table of the left side and probe the right side (ignoring why we would +/// choose this order, typically it depends on the size of each table, +/// etc.). +/// +/// ```text +/// ┌─────────────────────┐ +/// │ FilterExec │ +/// │ filters = │ +/// │ [d.size > 100] │ +/// └─────────────────────┘ +/// │ +/// │ +/// ┌──────────▼──────────┐ +/// │ │ +/// │ HashJoinExec │ +/// │ [u.dept@hash(d.id)] │ +/// │ │ +/// └─────────────────────┘ +/// │ +/// ┌────────────┴────────────┐ +/// ┌──────────▼──────────┐ ┌──────────▼──────────┐ +/// │ DataSourceExec │ │ DataSourceExec │ +/// │ alias [users as u] │ │ alias [dept as d] │ +/// │ │ │ │ +/// └─────────────────────┘ └─────────────────────┘ +/// ``` +/// +/// There are two pushdowns we can do here: +/// 1. Push down the `d.size > 100` filter through the `HashJoinExec` node to the `DataSourceExec` +/// node for the `departments` table. +/// 2. Push down the hash table state from the `HashJoinExec` node to the `DataSourceExec` node to avoid reading +/// rows from the `users` table that will be eliminated by the join. +/// This can be done via a bloom filter or similar and is not (yet) supported +/// in DataFusion. See . +/// +/// ```text +/// ┌─────────────────────┐ +/// │ │ +/// │ HashJoinExec │ +/// │ [u.dept@hash(d.id)] │ +/// │ │ +/// └─────────────────────┘ +/// │ +/// ┌────────────┴────────────┐ +/// ┌──────────▼──────────┐ ┌──────────▼──────────┐ +/// │ DataSourceExec │ │ DataSourceExec │ +/// │ alias [users as u] │ │ alias [dept as d] │ +/// │ filters = │ │ filters = │ +/// │ [depg@hash(d.id)] │ │ [ d.size > 100] │ +/// └─────────────────────┘ └─────────────────────┘ +/// ``` +/// +/// You may notice in this case that the filter is *dynamic*: the hash table +/// is built _after_ the `departments` table is read and at runtime. We +/// don't have a concrete `InList` filter or similar to push down at +/// optimization time. These sorts of dynamic filters are handled by +/// building a specialized [`PhysicalExpr`] that can be evaluated at runtime +/// and internally maintains a reference to the hash table or other state. +/// +/// To make working with these sorts of dynamic filters more tractable we have the method `PhysicalExpr::snapshot` +/// (TODO: add reference after is merged) +/// which attempts to simplify a dynamic filter into a "basic" non-dynamic filter. +/// For a join this could mean converting it to an `InList` filter or a min/max filter for example. +/// See `datafusion/physical-plan/src/dynamic_filters.rs` for more details. +/// +/// # Example: Push TopK filters into Scans +/// +/// Another form of dynamic filter is pushing down the state of a `TopK` +/// operator for queries like `SELECT * FROM t ORDER BY id LIMIT 10`: +/// +/// ```text +/// ┌──────────────────────┐ +/// │ TopK │ +/// │ limit = 10 │ +/// │ order by = [id] │ +/// └──────────────────────┘ +/// │ +/// ▼ +/// ┌──────────────────────┐ +/// │ DataSourceExec │ +/// │ projection = * │ +/// └──────────────────────┘ +/// ``` +/// +/// We can avoid large amounts of data processing by transforming this into: +/// +/// ```text +/// ┌──────────────────────┐ +/// │ TopK │ +/// │ limit = 10 │ +/// │ order by = [id] │ +/// └──────────────────────┘ +/// │ +/// ▼ +/// ┌──────────────────────┐ +/// │ DataSourceExec │ +/// │ projection = * │ +/// │ filters = │ +/// │ [id < @ TopKHeap] │ +/// └──────────────────────┘ +/// ``` +/// +/// Now as we fill our `TopK` heap we can push down the state of the heap to +/// the `DataSourceExec` node to avoid reading files / row groups / pages / +/// rows that could not possibly be in the top 10. +/// +/// This is not yet implemented in DataFusion. See +/// +/// +/// [`FilterExec`]: datafusion_physical_plan::filter::FilterExec +/// [`ProjectionExec`]: datafusion_phyiscal_plan::projection::ProjectionExec +/// [`AggregateExec`]: datafusion_phyiscal_plan::aggregates::AggregateExec #[derive(Debug)] pub struct PushdownFilter {} @@ -58,7 +398,7 @@ impl PhysicalOptimizerRule for PushdownFilter { context .transform_down(|mut node| { let (mut child_filters, remaining_filters, plan) = - match node.plan.try_pushdown_filters( + match Arc::clone(&node.plan).try_pushdown_filters( FilterDescription { filters: node.data.take_filters(), }, diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index b20e6c116c60c..39846b9e160e3 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -216,7 +216,7 @@ impl ExecutionPlan for CoalesceBatchesExec { } fn try_pushdown_filters( - &self, + self: Arc, fd: FilterDescription, _config: &ConfigOptions, ) -> Result>> { @@ -225,7 +225,7 @@ impl ExecutionPlan for CoalesceBatchesExec { Ok(FilterPushdownSupport::Supported { child_filters, remaining_filters, - op: Arc::new(self.clone()), + op: Arc::clone(&self) as Arc, }) } } diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index 8795ad878a8a2..068f9f5a83e29 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -469,8 +469,19 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { Ok(None) } + /// Attempts to recursively push given filters from the top of the tree into leafs. + /// + /// # Default Implementation + /// + /// The default implementation assumes: + /// * Parent filters can't be passed onto children. + /// * This node has no filters to contribute. + /// + /// See [`PushdownFilter`] for more details. + /// + /// [`PushdownFilter`]: datafusion_physical_optimizer::filter_pushdown::PushdownFilter fn try_pushdown_filters( - &self, + self: Arc, fd: FilterDescription, _config: &ConfigOptions, ) -> Result>> { diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 238e065a37a98..c6ef6cdd393a7 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -50,10 +50,10 @@ use datafusion_expr::Operator; use datafusion_physical_expr::equivalence::ProjectionMapping; use datafusion_physical_expr::expressions::BinaryExpr; use datafusion_physical_expr::intervals::utils::check_support; -use datafusion_physical_expr::utils::collect_columns; +use datafusion_physical_expr::utils::{collect_columns, reassign_predicate_columns}; use datafusion_physical_expr::{ - analyze, conjunction, split_conjunction, AcrossPartitions, AnalysisContext, - ConstExpr, ExprBoundaries, PhysicalExpr, PhysicalExprRef, + analyze, split_conjunction, AcrossPartitions, AnalysisContext, ConstExpr, + ExprBoundaries, PhysicalExpr, }; use datafusion_physical_expr_common::physical_expr::fmt_sql; @@ -437,11 +437,18 @@ impl ExecutionPlan for FilterExec { } fn try_pushdown_filters( - &self, + self: Arc, mut fd: FilterDescription, _config: &ConfigOptions, ) -> Result>> { - fd.filters.push(Arc::clone(self.predicate())); + // filters are in terms of the output columns of this plan, need to adapt them if we have a projection + let predicate = if self.projection.is_some() { + let input_schema = self.input.schema(); + reassign_predicate_columns(Arc::clone(&self.predicate), &input_schema, false)? + } else { + Arc::clone(&self.predicate) + }; + fd.filters.push(predicate); let child_filters = vec![fd]; let remaining_filters = FilterDescription { filters: vec![] }; Ok(FilterPushdownSupport::Supported { diff --git a/datafusion/physical-plan/src/filter_pushdown.rs b/datafusion/physical-plan/src/filter_pushdown.rs index 26ac480c6775b..4873f64a7f961 100644 --- a/datafusion/physical-plan/src/filter_pushdown.rs +++ b/datafusion/physical-plan/src/filter_pushdown.rs @@ -17,9 +17,6 @@ use std::sync::Arc; -use crate::ExecutionPlan; - -use datafusion_common::Result; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; #[derive(Default)] diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 50aa7a4605452..d57bece39a1d9 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -727,7 +727,7 @@ impl ExecutionPlan for RepartitionExec { } fn try_pushdown_filters( - &self, + self: Arc, fd: FilterDescription, _config: &ConfigOptions, ) -> Result>> { @@ -736,7 +736,7 @@ impl ExecutionPlan for RepartitionExec { Ok(FilterPushdownSupport::Supported { child_filters, remaining_filters, - op: Arc::new(self.clone()), + op: Arc::clone(&self) as Arc, }) } } From e4d8a8ccb4192162e6a0065c7029d5082dd57d34 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Wed, 16 Apr 2025 12:05:04 +0300 Subject: [PATCH 24/40] dbg --- .../physical_optimizer/filter_pushdown.rs | 31 ++-- datafusion/datasource/src/file.rs | 8 +- datafusion/datasource/src/file_scan_config.rs | 39 +++-- datafusion/datasource/src/source.rs | 44 ++++-- .../physical-optimizer/src/filter_pushdown.rs | 143 ++++++++++++------ .../physical-plan/src/coalesce_batches.rs | 24 +-- .../physical-plan/src/execution_plan.rs | 15 +- datafusion/physical-plan/src/filter.rs | 38 +++-- .../physical-plan/src/filter_pushdown.rs | 41 ++++- .../physical-plan/src/repartition/mod.rs | 24 +-- 10 files changed, 268 insertions(+), 139 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs index 12bbb480e9161..3d3da279e40fa 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs @@ -41,7 +41,8 @@ use datafusion_physical_expr_common::physical_expr::fmt_sql; use datafusion_physical_optimizer::filter_pushdown::PushdownFilter; use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::filter_pushdown::{ - FilterDescription, FilterPushdownSupport, + filter_pushdown_not_supported, FilterDescription, FilterPushdownResult, + FilterPushdownSupport, }; use datafusion_physical_plan::{ aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}, @@ -150,25 +151,23 @@ impl FileSource for TestSource { &self, fd: FilterDescription, config: &ConfigOptions, - ) -> Result>> { - if self.support { - if config.execution.parquet.pushdown_filters { - return Ok(FilterPushdownSupport::Supported { - child_filters: vec![], - remaining_filters: FilterDescription { filters: vec![] }, + ) -> Result>> { + if self.support && config.execution.parquet.pushdown_filters { + Ok(FilterPushdownResult { + support: FilterPushdownSupport::Supported { + child_descriptions: vec![], op: Arc::new(TestSource { - support: self.support, + support: true, predicate: Some(conjunction(fd.filters)), - statistics: self.statistics.clone(), + statistics: self.statistics.clone(), // should be updated ? }), - }); - } + retry: false, + }, + remaining_description: Default::default(), + }) + } else { + Ok(filter_pushdown_not_supported(fd)) } - Ok(FilterPushdownSupport::Supported { - child_filters: vec![], - remaining_filters: fd, - op: Arc::new(self.clone()), - }) } } diff --git a/datafusion/datasource/src/file.rs b/datafusion/datasource/src/file.rs index 8508bbb9dc7b9..49784a5547144 100644 --- a/datafusion/datasource/src/file.rs +++ b/datafusion/datasource/src/file.rs @@ -30,7 +30,8 @@ use datafusion_common::config::ConfigOptions; use datafusion_common::{Result, Statistics}; use datafusion_physical_expr::LexOrdering; use datafusion_physical_plan::filter_pushdown::{ - FilterDescription, FilterPushdownSupport, + filter_pushdown_not_supported, FilterDescription, FilterPushdownResult, + FilterPushdownSupport, }; use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; use datafusion_physical_plan::DisplayFormatType; @@ -97,6 +98,7 @@ pub trait FileSource: Send + Sync { } Ok(None) } + /// Try to push down filters into this FileSource. /// See [`ExecutionPlan::try_pushdown_filters`] for more details. /// @@ -105,7 +107,7 @@ pub trait FileSource: Send + Sync { &self, fd: FilterDescription, _config: &ConfigOptions, - ) -> Result>> { - Ok(FilterPushdownSupport::NotSupported(fd)) + ) -> Result>> { + Ok(filter_pushdown_not_supported(fd)) } } diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index 667db9f164e64..8cb07d06dad70 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -53,7 +53,8 @@ use datafusion_physical_expr::{ PhysicalSortExpr, }; use datafusion_physical_plan::filter_pushdown::{ - FilterDescription, FilterPushdownSupport, + filter_pushdown_not_supported, FilterDescription, FilterPushdownResult, + FilterPushdownSupport, }; use datafusion_physical_plan::{ display::{display_orderings, ProjectSchemaDisplay}, @@ -593,26 +594,38 @@ impl DataSource for FileScanConfig { &self, fd: FilterDescription, config: &ConfigOptions, - ) -> Result>> { - match self.file_source.try_pushdown_filters(fd, config)? { + ) -> Result>> { + let FilterPushdownResult { + support, + remaining_description, + } = self.file_source.try_pushdown_filters(fd, config)?; + + match support { FilterPushdownSupport::Supported { - child_filters, - remaining_filters, - op: file_source, + child_descriptions, + op, + retry, } => { let new_data_source = Arc::new( FileScanConfigBuilder::from(self.clone()) - .with_source(file_source) + .with_source(op) .build(), ); - Ok(FilterPushdownSupport::Supported { - child_filters, - remaining_filters, - op: new_data_source, + + debug_assert!(child_descriptions.is_empty()); + debug_assert!(!retry); + + Ok(FilterPushdownResult { + support: FilterPushdownSupport::Supported { + child_descriptions, + op: new_data_source, + retry, + }, + remaining_description, }) } - FilterPushdownSupport::NotSupported(fd) => { - Ok(FilterPushdownSupport::NotSupported(fd)) + FilterPushdownSupport::NotSupported => { + Ok(filter_pushdown_not_supported(remaining_description)) } } } diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index 2a087aaff9cc3..12a8f16ebd6e3 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -36,7 +36,8 @@ use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_physical_expr::{EquivalenceProperties, Partitioning}; use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::filter_pushdown::{ - FilterDescription, FilterPushdownSupport, + filter_pushdown_not_supported, FilterDescription, FilterPushdownResult, + FilterPushdownSupport, }; /// Common behaviors in Data Sources for both from Files and Memory. @@ -88,8 +89,8 @@ pub trait DataSource: Send + Sync + Debug { &self, fd: FilterDescription, _config: &ConfigOptions, - ) -> Result>> { - Ok(FilterPushdownSupport::NotSupported(fd)) + ) -> Result>> { + Ok(filter_pushdown_not_supported(fd)) } } @@ -206,26 +207,37 @@ impl ExecutionPlan for DataSourceExec { } fn try_pushdown_filters( - self: Arc, + &self, fd: FilterDescription, config: &ConfigOptions, - ) -> Result>> { - let mut exec = Arc::unwrap_or_clone(self); - match exec.data_source.try_pushdown_filters(fd, config)? { + ) -> Result>> { + let FilterPushdownResult { + support, + remaining_description, + } = self.data_source.try_pushdown_filters(fd, config)?; + + match support { FilterPushdownSupport::Supported { - child_filters, - remaining_filters, + child_descriptions, op, + retry, } => { - exec.data_source = op; - Ok(FilterPushdownSupport::Supported { - child_filters, - remaining_filters, - op: Arc::new(exec) as Arc, + let new_exec = Arc::new(DataSourceExec::new(op)); + + debug_assert!(child_descriptions.is_empty()); + debug_assert!(!retry); + + Ok(FilterPushdownResult { + support: FilterPushdownSupport::Supported { + child_descriptions, + op: new_exec, + retry, + }, + remaining_description, }) } - FilterPushdownSupport::NotSupported(fd) => { - Ok(FilterPushdownSupport::NotSupported(fd)) + FilterPushdownSupport::NotSupported => { + Ok(filter_pushdown_not_supported(remaining_description)) } } } diff --git a/datafusion/physical-optimizer/src/filter_pushdown.rs b/datafusion/physical-optimizer/src/filter_pushdown.rs index c076b397cfd41..d0e5b555a6361 100644 --- a/datafusion/physical-optimizer/src/filter_pushdown.rs +++ b/datafusion/physical-optimizer/src/filter_pushdown.rs @@ -23,7 +23,7 @@ use datafusion_common::{config::ConfigOptions, Result}; use datafusion_physical_expr::conjunction; use datafusion_physical_plan::filter::FilterExec; use datafusion_physical_plan::filter_pushdown::{ - FilterDescription, FilterPushdownSupport, + FilterDescription, FilterPushdownResult, FilterPushdownSupport, }; use datafusion_physical_plan::tree_node::PlanContext; use datafusion_physical_plan::ExecutionPlan; @@ -31,10 +31,10 @@ use datafusion_physical_plan::ExecutionPlan; /// Attempts to recursively push given filters from the top of the tree into leafs. /// /// # Default Implementation -/// +/// /// The default implementation in [`ExecutionPlan::try_pushdown_filters`] is a no-op /// that assumes that: -/// +/// /// * Parent filters can't be passed onto children. /// * This node has no filters to contribute. /// @@ -379,12 +379,6 @@ impl Default for PushdownFilter { } } -impl PushdownFilter { - pub fn new() -> Self { - Self {} - } -} - pub type FilterDescriptionContext = PlanContext; impl PhysicalOptimizerRule for PushdownFilter { @@ -396,41 +390,7 @@ impl PhysicalOptimizerRule for PushdownFilter { let context = FilterDescriptionContext::new_default(plan); context - .transform_down(|mut node| { - let (mut child_filters, remaining_filters, plan) = - match Arc::clone(&node.plan).try_pushdown_filters( - FilterDescription { - filters: node.data.take_filters(), - }, - config, - )? { - FilterPushdownSupport::Supported { - child_filters, - remaining_filters, - op: plan, - } => (child_filters, remaining_filters, plan), - FilterPushdownSupport::NotSupported(fd) => { - (vec![], fd, Arc::clone(&node.plan)) - } - }; - - if remaining_filters.filters.is_empty() { - node = FilterDescriptionContext::new_default(plan); - for (child, filter) in node.children.iter_mut().zip(child_filters) { - child.data = filter; - } - } else { - let mut new_child_node = FilterDescriptionContext::new_default(plan); - new_child_node.data = child_filters.swap_remove(0); - node.plan = Arc::new(FilterExec::try_new( - conjunction(remaining_filters.filters), - Arc::clone(&new_child_node.plan), - )?); - node.children = vec![new_child_node]; - node.data = FilterDescription::default(); - } - Ok(Transformed::yes(node)) - }) + .transform_down(|node| Self::try_pushdown(node, config)) .map(|updated| updated.data.plan) } @@ -442,3 +402,98 @@ impl PhysicalOptimizerRule for PushdownFilter { true // Filter pushdown does not change the schema of the plan } } + +impl PushdownFilter { + pub fn new() -> Self { + Self {} + } + + fn try_pushdown( + mut node: FilterDescriptionContext, + config: &ConfigOptions, + ) -> Result> { + let initial_plan = Arc::clone(&node.plan); + let initial_description = FilterDescription { + filters: node.data.take_filters(), + }; + + let FilterPushdownResult { + support, + remaining_description, + } = initial_plan.try_pushdown_filters(initial_description, config)?; + + match support { + FilterPushdownSupport::Supported { + mut child_descriptions, + op, + retry, + } => { + if retry { + // This check handles cases where the current operator is entirely removed + // from the plan and replaced with its child. In such cases, to not skip + // over the new node, we need to explicitly re-apply this pushdown logic + // to the new node. + // + // TODO: If TreeNodeRecursion supports a retry mechanism in the future, + // this manual recursion could be removed. + + // If the operator is removed, it should not leave any filters as remaining + debug_assert!(remaining_description.filters.is_empty()); + node.plan = op; + // Operators having 2 children cannot be removed + node.data = child_descriptions.swap_remove(0); + node.children = node.children.swap_remove(0).children; + Self::try_pushdown(node, config) + } else { + if remaining_description.filters.is_empty() { + node.plan = op; + for (child, descr) in + node.children.iter_mut().zip(child_descriptions) + { + child.data = descr; + } + } else { + node = insert_filter_exec( + node, + child_descriptions, + remaining_description, + )?; + } + Ok(Transformed::yes(node)) + } + } + FilterPushdownSupport::NotSupported => { + let children_len = node.children.len(); + node = insert_filter_exec( + node, + vec![FilterDescription::default(); children_len], + remaining_description, + )?; + Ok(Transformed::yes(node)) + } + } + } +} + +fn insert_filter_exec( + node: FilterDescriptionContext, + mut child_descriptions: Vec, + remaining_description: FilterDescription, +) -> Result { + let mut new_child_node = node; + + // Filter has one child + new_child_node.data = child_descriptions.swap_remove(0); + let new_plan = Arc::new(FilterExec::try_new( + conjunction(remaining_description.filters), + Arc::clone(&new_child_node.plan), + )?); + let new_children = vec![new_child_node]; + let new_data = FilterDescription::default(); + + Ok(FilterDescriptionContext::new( + new_plan, + new_data, + new_children, + )) +} diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index 39846b9e160e3..3ff20abfede7b 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -35,7 +35,9 @@ use datafusion_execution::TaskContext; use crate::coalesce::{BatchCoalescer, CoalescerState}; use crate::execution_plan::CardinalityEffect; -use crate::filter_pushdown::{FilterDescription, FilterPushdownSupport}; +use crate::filter_pushdown::{ + FilterDescription, FilterPushdownResult, FilterPushdownSupport, +}; use datafusion_common::config::ConfigOptions; use futures::ready; use futures::stream::{Stream, StreamExt}; @@ -216,16 +218,20 @@ impl ExecutionPlan for CoalesceBatchesExec { } fn try_pushdown_filters( - self: Arc, + &self, fd: FilterDescription, _config: &ConfigOptions, - ) -> Result>> { - let child_filters = vec![fd]; - let remaining_filters = FilterDescription::default(); - Ok(FilterPushdownSupport::Supported { - child_filters, - remaining_filters, - op: Arc::clone(&self) as Arc, + ) -> Result>> { + let child_descriptions = vec![fd]; + let remaining_description = FilterDescription::empty(); + + Ok(FilterPushdownResult { + support: FilterPushdownSupport::Supported { + child_descriptions, + op: Arc::new(self.clone()), + retry: false, + }, + remaining_description, }) } } diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index 068f9f5a83e29..1969c454de8e9 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -16,7 +16,10 @@ // under the License. pub use crate::display::{DefaultDisplay, DisplayAs, DisplayFormatType, VerboseDisplay}; -use crate::filter_pushdown::{FilterDescription, FilterPushdownSupport}; +use crate::filter_pushdown::{ + filter_pushdown_not_supported, FilterDescription, FilterPushdownResult, + FilterPushdownSupport, +}; pub use crate::metrics::Metric; pub use crate::ordering::InputOrderMode; pub use crate::stream::EmptyRecordBatchStream; @@ -476,16 +479,16 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// The default implementation assumes: /// * Parent filters can't be passed onto children. /// * This node has no filters to contribute. - /// + /// /// See [`PushdownFilter`] for more details. - /// + /// /// [`PushdownFilter`]: datafusion_physical_optimizer::filter_pushdown::PushdownFilter fn try_pushdown_filters( - self: Arc, + &self, fd: FilterDescription, _config: &ConfigOptions, - ) -> Result>> { - Ok(FilterPushdownSupport::NotSupported(fd)) + ) -> Result>> { + Ok(filter_pushdown_not_supported(fd)) } } diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index c6ef6cdd393a7..d4a6d2cec4a88 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -26,7 +26,9 @@ use super::{ }; use crate::common::can_project; use crate::execution_plan::CardinalityEffect; -use crate::filter_pushdown::{FilterDescription, FilterPushdownSupport}; +use crate::filter_pushdown::{ + FilterDescription, FilterPushdownResult, FilterPushdownSupport, +}; use crate::projection::{ make_with_child, try_embed_projection, update_expr, EmbeddedProjection, ProjectionExec, @@ -437,24 +439,28 @@ impl ExecutionPlan for FilterExec { } fn try_pushdown_filters( - self: Arc, + &self, mut fd: FilterDescription, _config: &ConfigOptions, - ) -> Result>> { - // filters are in terms of the output columns of this plan, need to adapt them if we have a projection - let predicate = if self.projection.is_some() { - let input_schema = self.input.schema(); - reassign_predicate_columns(Arc::clone(&self.predicate), &input_schema, false)? - } else { - Arc::clone(&self.predicate) + ) -> Result>> { + if self.projection.is_some() { + return Ok(FilterPushdownResult { + support: FilterPushdownSupport::NotSupported, + remaining_description: fd, + }); }; - fd.filters.push(predicate); - let child_filters = vec![fd]; - let remaining_filters = FilterDescription { filters: vec![] }; - Ok(FilterPushdownSupport::Supported { - child_filters, - remaining_filters, - op: Arc::clone(&self.input), + + fd.filters.push(self.predicate.clone()); + let child_descriptions = vec![fd]; + let remaining_description = FilterDescription { filters: vec![] }; + + Ok(FilterPushdownResult { + support: FilterPushdownSupport::Supported { + child_descriptions, + op: Arc::new(self.clone()), + retry: false, + }, + remaining_description, }) } } diff --git a/datafusion/physical-plan/src/filter_pushdown.rs b/datafusion/physical-plan/src/filter_pushdown.rs index 4873f64a7f961..dd85f91826a6b 100644 --- a/datafusion/physical-plan/src/filter_pushdown.rs +++ b/datafusion/physical-plan/src/filter_pushdown.rs @@ -19,30 +19,57 @@ use std::sync::Arc; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; -#[derive(Default)] +#[derive(Clone)] pub struct FilterDescription { /// Expressions coming from the parent nodes pub filters: Vec>, } +impl Default for FilterDescription { + fn default() -> Self { + Self::empty() + } +} + impl FilterDescription { /// Takes the filters out of the struct, leaving an empty vector in its place. pub fn take_filters(&mut self) -> Vec> { std::mem::take(&mut self.filters) } + + pub fn empty() -> FilterDescription { + Self { filters: vec![] } + } } pub enum FilterPushdownSupport { Supported { // Filter predicates which can be pushed down through the operator. // NOTE that these are not placed into any operator. - child_filters: Vec, - // Filters which cannot be pushed down through the operator. - // NOTE that caller of try_pushdown_filters() should handle these remanining predicates, - // possibly introducing a FilterExec on top of this operator. - remaining_filters: FilterDescription, + child_descriptions: Vec, // Possibly updated new operator op: T, + // Whether the node is removed from the plan and the rule should be re-run manually + // on the new node. + // TODO: If TreeNodeRecursion supports Retry mechanism, this flag can be removed + retry: bool, }, - NotSupported(FilterDescription), + NotSupported, +} + +pub struct FilterPushdownResult { + pub support: FilterPushdownSupport, + // Filters which cannot be pushed down through the operator. + // NOTE that caller of try_pushdown_filters() should handle these remanining predicates, + // possibly introducing a FilterExec on top of this operator. + pub remaining_description: FilterDescription, +} + +pub fn filter_pushdown_not_supported( + remaining_description: FilterDescription, +) -> FilterPushdownResult { + FilterPushdownResult { + support: FilterPushdownSupport::NotSupported, + remaining_description, + } } diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index d57bece39a1d9..da47c2896eb49 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -53,7 +53,9 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::{EquivalenceProperties, PhysicalExpr}; use datafusion_physical_expr_common::sort_expr::LexOrdering; -use crate::filter_pushdown::{FilterDescription, FilterPushdownSupport}; +use crate::filter_pushdown::{ + FilterDescription, FilterPushdownResult, FilterPushdownSupport, +}; use futures::stream::Stream; use futures::{FutureExt, StreamExt, TryStreamExt}; use log::trace; @@ -727,16 +729,20 @@ impl ExecutionPlan for RepartitionExec { } fn try_pushdown_filters( - self: Arc, + &self, fd: FilterDescription, _config: &ConfigOptions, - ) -> Result>> { - let child_filters = vec![fd]; - let remaining_filters = FilterDescription::default(); - Ok(FilterPushdownSupport::Supported { - child_filters, - remaining_filters, - op: Arc::clone(&self) as Arc, + ) -> Result>> { + let child_descriptions = vec![fd]; + let remaining_description = FilterDescription::empty(); + + Ok(FilterPushdownResult { + support: FilterPushdownSupport::Supported { + child_descriptions, + op: Arc::new(self.clone()), + retry: false, + }, + remaining_description, }) } } From 3ec1b2a4b91a7fcbc1ad476da02a9af65c1cb0fd Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Wed, 16 Apr 2025 15:12:16 +0300 Subject: [PATCH 25/40] dbg 2 --- .../physical-optimizer/src/filter_pushdown.rs | 33 ++++++++++++------- datafusion/physical-plan/src/filter.rs | 2 +- .../physical-plan/src/filter_pushdown.rs | 2 +- 3 files changed, 24 insertions(+), 13 deletions(-) diff --git a/datafusion/physical-optimizer/src/filter_pushdown.rs b/datafusion/physical-optimizer/src/filter_pushdown.rs index d0e5b555a6361..39b31a01bd5fe 100644 --- a/datafusion/physical-optimizer/src/filter_pushdown.rs +++ b/datafusion/physical-optimizer/src/filter_pushdown.rs @@ -414,7 +414,7 @@ impl PushdownFilter { ) -> Result> { let initial_plan = Arc::clone(&node.plan); let initial_description = FilterDescription { - filters: node.data.take_filters(), + filters: node.data.take_description(), }; let FilterPushdownResult { @@ -442,8 +442,13 @@ impl PushdownFilter { node.plan = op; // Operators having 2 children cannot be removed node.data = child_descriptions.swap_remove(0); - node.children = node.children.swap_remove(0).children; - Self::try_pushdown(node, config) + if !node.children.is_empty() { + node.children = node.children.swap_remove(0).children; + Self::try_pushdown(node, config) + } else { + node.children = vec![]; + Ok(Transformed::yes(node)) + } } else { if remaining_description.filters.is_empty() { node.plan = op; @@ -463,13 +468,17 @@ impl PushdownFilter { } } FilterPushdownSupport::NotSupported => { - let children_len = node.children.len(); - node = insert_filter_exec( - node, - vec![FilterDescription::default(); children_len], - remaining_description, - )?; - Ok(Transformed::yes(node)) + if remaining_description.filters.is_empty() { + Ok(Transformed::no(node)) + } else { + let children_len = node.children.len(); + node = insert_filter_exec( + node, + vec![FilterDescription::default(); children_len], + remaining_description, + )?; + Ok(Transformed::yes(node)) + } } } } @@ -483,7 +492,9 @@ fn insert_filter_exec( let mut new_child_node = node; // Filter has one child - new_child_node.data = child_descriptions.swap_remove(0); + if !child_descriptions.is_empty() { + new_child_node.data = child_descriptions.swap_remove(0); + } let new_plan = Arc::new(FilterExec::try_new( conjunction(remaining_description.filters), Arc::clone(&new_child_node.plan), diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index d4a6d2cec4a88..fcc5a6de48f61 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -458,7 +458,7 @@ impl ExecutionPlan for FilterExec { support: FilterPushdownSupport::Supported { child_descriptions, op: Arc::new(self.clone()), - retry: false, + retry: true, }, remaining_description, }) diff --git a/datafusion/physical-plan/src/filter_pushdown.rs b/datafusion/physical-plan/src/filter_pushdown.rs index dd85f91826a6b..6ef6149e94990 100644 --- a/datafusion/physical-plan/src/filter_pushdown.rs +++ b/datafusion/physical-plan/src/filter_pushdown.rs @@ -33,7 +33,7 @@ impl Default for FilterDescription { impl FilterDescription { /// Takes the filters out of the struct, leaving an empty vector in its place. - pub fn take_filters(&mut self) -> Vec> { + pub fn take_description(&mut self) -> Vec> { std::mem::take(&mut self.filters) } From a2df5e0438aa1f810d1cb990607ae0d574065184 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Wed, 16 Apr 2025 07:31:52 -0500 Subject: [PATCH 26/40] avoid clones --- datafusion/datasource/src/file.rs | 1 - datafusion/datasource/src/source.rs | 1 + datafusion/physical-optimizer/src/filter_pushdown.rs | 6 +++++- datafusion/physical-plan/src/coalesce_batches.rs | 3 ++- datafusion/physical-plan/src/execution_plan.rs | 2 +- datafusion/physical-plan/src/filter.rs | 5 +++-- datafusion/physical-plan/src/repartition/mod.rs | 3 ++- 7 files changed, 14 insertions(+), 7 deletions(-) diff --git a/datafusion/datasource/src/file.rs b/datafusion/datasource/src/file.rs index 49784a5547144..835285b21e38a 100644 --- a/datafusion/datasource/src/file.rs +++ b/datafusion/datasource/src/file.rs @@ -31,7 +31,6 @@ use datafusion_common::{Result, Statistics}; use datafusion_physical_expr::LexOrdering; use datafusion_physical_plan::filter_pushdown::{ filter_pushdown_not_supported, FilterDescription, FilterPushdownResult, - FilterPushdownSupport, }; use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; use datafusion_physical_plan::DisplayFormatType; diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index 12a8f16ebd6e3..ce8ba81a41676 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -208,6 +208,7 @@ impl ExecutionPlan for DataSourceExec { fn try_pushdown_filters( &self, + _node: Arc, fd: FilterDescription, config: &ConfigOptions, ) -> Result>> { diff --git a/datafusion/physical-optimizer/src/filter_pushdown.rs b/datafusion/physical-optimizer/src/filter_pushdown.rs index d0e5b555a6361..5ec210f60cc35 100644 --- a/datafusion/physical-optimizer/src/filter_pushdown.rs +++ b/datafusion/physical-optimizer/src/filter_pushdown.rs @@ -420,7 +420,11 @@ impl PushdownFilter { let FilterPushdownResult { support, remaining_description, - } = initial_plan.try_pushdown_filters(initial_description, config)?; + } = initial_plan.try_pushdown_filters( + Arc::clone(&initial_plan), + initial_description, + config, + )?; match support { FilterPushdownSupport::Supported { diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index 3ff20abfede7b..a73a674081f6e 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -219,6 +219,7 @@ impl ExecutionPlan for CoalesceBatchesExec { fn try_pushdown_filters( &self, + node: Arc, fd: FilterDescription, _config: &ConfigOptions, ) -> Result>> { @@ -228,7 +229,7 @@ impl ExecutionPlan for CoalesceBatchesExec { Ok(FilterPushdownResult { support: FilterPushdownSupport::Supported { child_descriptions, - op: Arc::new(self.clone()), + op: Arc::clone(&node), retry: false, }, remaining_description, diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index 1969c454de8e9..1dd9cb1e2ea8e 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -18,7 +18,6 @@ pub use crate::display::{DefaultDisplay, DisplayAs, DisplayFormatType, VerboseDisplay}; use crate::filter_pushdown::{ filter_pushdown_not_supported, FilterDescription, FilterPushdownResult, - FilterPushdownSupport, }; pub use crate::metrics::Metric; pub use crate::ordering::InputOrderMode; @@ -485,6 +484,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// [`PushdownFilter`]: datafusion_physical_optimizer::filter_pushdown::PushdownFilter fn try_pushdown_filters( &self, + _node: Arc, fd: FilterDescription, _config: &ConfigOptions, ) -> Result>> { diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index d4a6d2cec4a88..aafbec23d1130 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -52,7 +52,7 @@ use datafusion_expr::Operator; use datafusion_physical_expr::equivalence::ProjectionMapping; use datafusion_physical_expr::expressions::BinaryExpr; use datafusion_physical_expr::intervals::utils::check_support; -use datafusion_physical_expr::utils::{collect_columns, reassign_predicate_columns}; +use datafusion_physical_expr::utils::collect_columns; use datafusion_physical_expr::{ analyze, split_conjunction, AcrossPartitions, AnalysisContext, ConstExpr, ExprBoundaries, PhysicalExpr, @@ -440,6 +440,7 @@ impl ExecutionPlan for FilterExec { fn try_pushdown_filters( &self, + node: Arc, mut fd: FilterDescription, _config: &ConfigOptions, ) -> Result>> { @@ -457,7 +458,7 @@ impl ExecutionPlan for FilterExec { Ok(FilterPushdownResult { support: FilterPushdownSupport::Supported { child_descriptions, - op: Arc::new(self.clone()), + op: Arc::clone(&node), retry: false, }, remaining_description, diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index da47c2896eb49..375cdd7eb2227 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -730,6 +730,7 @@ impl ExecutionPlan for RepartitionExec { fn try_pushdown_filters( &self, + node: Arc, fd: FilterDescription, _config: &ConfigOptions, ) -> Result>> { @@ -739,7 +740,7 @@ impl ExecutionPlan for RepartitionExec { Ok(FilterPushdownResult { support: FilterPushdownSupport::Supported { child_descriptions, - op: Arc::new(self.clone()), + op: Arc::clone(&node), retry: false, }, remaining_description, From 6938d528b1a1c4fd50095cb3c76834f0c55ca1bb Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Wed, 16 Apr 2025 15:33:57 +0300 Subject: [PATCH 27/40] part 3 --- .../core/tests/physical_optimizer/filter_pushdown.rs | 6 ++++-- datafusion/physical-optimizer/src/filter_pushdown.rs | 8 ++++---- datafusion/physical-plan/src/filter.rs | 2 +- datafusion/physical-plan/src/filter_pushdown.rs | 4 +++- 4 files changed, 12 insertions(+), 8 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs index 3d3da279e40fa..1a315c3b95b90 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs @@ -227,7 +227,7 @@ fn test_pushdown_into_scan_with_config_options() { output: Ok: - FilterExec: a@0 = foo - - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, predicate=a@0 = foo + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test " ); @@ -467,7 +467,9 @@ impl OptimizationTest { where O: PhysicalOptimizerRule, { - Self::new_with_config(input_plan, opt, &ConfigOptions::default()) + let mut parquet_pushdown_config = ConfigOptions::default(); + parquet_pushdown_config.execution.parquet.pushdown_filters = true; + Self::new_with_config(input_plan, opt, &parquet_pushdown_config) } pub fn new_with_config( diff --git a/datafusion/physical-optimizer/src/filter_pushdown.rs b/datafusion/physical-optimizer/src/filter_pushdown.rs index 39b31a01bd5fe..ef2d17c8a832c 100644 --- a/datafusion/physical-optimizer/src/filter_pushdown.rs +++ b/datafusion/physical-optimizer/src/filter_pushdown.rs @@ -442,12 +442,12 @@ impl PushdownFilter { node.plan = op; // Operators having 2 children cannot be removed node.data = child_descriptions.swap_remove(0); - if !node.children.is_empty() { - node.children = node.children.swap_remove(0).children; - Self::try_pushdown(node, config) - } else { + if node.children.is_empty() { node.children = vec![]; Ok(Transformed::yes(node)) + } else { + node.children = node.children.swap_remove(0).children; + Self::try_pushdown(node, config) } } else { if remaining_description.filters.is_empty() { diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index fcc5a6de48f61..97f11a5af3cb7 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -457,7 +457,7 @@ impl ExecutionPlan for FilterExec { Ok(FilterPushdownResult { support: FilterPushdownSupport::Supported { child_descriptions, - op: Arc::new(self.clone()), + op: Arc::clone(self.input()), retry: true, }, remaining_description, diff --git a/datafusion/physical-plan/src/filter_pushdown.rs b/datafusion/physical-plan/src/filter_pushdown.rs index 6ef6149e94990..ba11aca704c51 100644 --- a/datafusion/physical-plan/src/filter_pushdown.rs +++ b/datafusion/physical-plan/src/filter_pushdown.rs @@ -19,7 +19,7 @@ use std::sync::Arc; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; -#[derive(Clone)] +#[derive(Clone, Debug)] pub struct FilterDescription { /// Expressions coming from the parent nodes pub filters: Vec>, @@ -42,6 +42,7 @@ impl FilterDescription { } } +#[derive(Debug)] pub enum FilterPushdownSupport { Supported { // Filter predicates which can be pushed down through the operator. @@ -57,6 +58,7 @@ pub enum FilterPushdownSupport { NotSupported, } +#[derive(Debug)] pub struct FilterPushdownResult { pub support: FilterPushdownSupport, // Filters which cannot be pushed down through the operator. From 6836dd45056681a6230e4f6176936ed939719d9e Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Wed, 16 Apr 2025 07:52:03 -0500 Subject: [PATCH 28/40] fix lint --- datafusion/core/tests/physical_optimizer/filter_pushdown.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs index 3d3da279e40fa..ea9743d4c37c5 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs @@ -310,7 +310,7 @@ fn test_filter_with_projection() { // add a test where the filter is on a column that isn't included in the output let projection = vec![1]; - let predicate = col_lit_predicate("a", "foo", &schema()); + let predicate = col_lit_predicate("a", "foo", schema()); let plan = Arc::new( FilterExec::try_new(predicate, scan) .unwrap() From 7e952835565ccf3e9cfab8d05ce65fd81bc92c29 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Wed, 16 Apr 2025 16:21:29 +0300 Subject: [PATCH 29/40] tests pass --- .../physical_optimizer/filter_pushdown.rs | 22 ++++++++----------- datafusion/core/tests/sql/explain_analyze.rs | 4 ++-- 2 files changed, 11 insertions(+), 15 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs index 173f7aaf9b110..da1ea3e506a0b 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs @@ -245,7 +245,8 @@ fn test_pushdown_into_scan_with_config_options() { - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test output: Ok: - - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, predicate=a@0 = foo + - FilterExec: a@0 = foo + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test " ); } @@ -291,21 +292,16 @@ fn test_filter_with_projection() { insta::assert_snapshot!( OptimizationTest::new(plan, PushdownFilter{}), - @r#" + @r" OptimizationTest: input: - FilterExec: a@1 = foo, projection=[b@1, a@0] - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test output: - Err: Internal error: Schema mismatch: - - Before: - Schema { fields: [Field { name: "b", data_type: Utf8, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }, Field { name: "a", data_type: Utf8, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }], metadata: {} } - - After: - Schema { fields: [Field { name: "a", data_type: Utf8, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }, Field { name: "b", data_type: Utf8, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }, Field { name: "c", data_type: Float64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }], metadata: {} }. - This was likely caused by a bug in DataFusion's code and we would welcome that you file an bug report in our issue tracker - "#, + Ok: + - FilterExec: a@1 = foo, projection=[b@1, a@0] + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, predicate=true + ", ); // add a test where the filter is on a column that isn't included in the output @@ -326,8 +322,8 @@ fn test_filter_with_projection() { - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test output: Ok: - - FilterExec: true, projection=[b@1] - - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, predicate=a@0 = foo + - FilterExec: a@0 = foo, projection=[b@1] + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, predicate=true " ); } diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index e8ef34c2afe70..cb71bd022bc6c 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -609,8 +609,8 @@ async fn test_physical_plan_display_indent() { " RepartitionExec: partitioning=Hash([c1@0], 9000), input_partitions=9000", " AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[max(aggregate_test_100.c12), min(aggregate_test_100.c12)]", " CoalesceBatchesExec: target_batch_size=4096", - " FilterExec: c12@1 < 10", - " RepartitionExec: partitioning=RoundRobinBatch(9000), input_partitions=1", + " RepartitionExec: partitioning=RoundRobinBatch(9000), input_partitions=1", + " FilterExec: c12@1 < 10", " DataSourceExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1, c12], file_type=csv, has_header=true", ]; From e2f8c124d4f280f513e910b00db37158c865bf90 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Wed, 16 Apr 2025 19:33:10 +0300 Subject: [PATCH 30/40] Update filter.rs --- datafusion/physical-plan/src/filter.rs | 57 ++++++++++++++++++-------- 1 file changed, 40 insertions(+), 17 deletions(-) diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 34f1550fa7e08..7f5a93fc0a567 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -50,7 +50,7 @@ use datafusion_common::{ use datafusion_execution::TaskContext; use datafusion_expr::Operator; use datafusion_physical_expr::equivalence::ProjectionMapping; -use datafusion_physical_expr::expressions::BinaryExpr; +use datafusion_physical_expr::expressions::{BinaryExpr, Column}; use datafusion_physical_expr::intervals::utils::check_support; use datafusion_physical_expr::utils::collect_columns; use datafusion_physical_expr::{ @@ -444,25 +444,48 @@ impl ExecutionPlan for FilterExec { mut fd: FilterDescription, _config: &ConfigOptions, ) -> Result>> { - if self.projection.is_some() { - return Ok(FilterPushdownResult { - support: FilterPushdownSupport::NotSupported, - remaining_description: fd, - }); - }; - - fd.filters.push(self.predicate.clone()); + fd.filters.push(Arc::clone(&self.predicate)); let child_descriptions = vec![fd]; let remaining_description = FilterDescription { filters: vec![] }; + let filter_child = Arc::clone(node.children()[0]); - Ok(FilterPushdownResult { - support: FilterPushdownSupport::Supported { - child_descriptions, - op: Arc::clone(self.input()), - retry: true, - }, - remaining_description, - }) + if self.projection.is_some() { + let filter_child_schema = filter_child.schema(); + let proj_exprs = self + .projection + .as_ref() + .unwrap() + .iter() + .map(|p| { + let col_name = filter_child_schema.field(*p).clone(); + ( + Arc::new(Column::new(col_name.name(), *p)) + as Arc, + col_name.name().to_string(), + ) + }) + .collect::>(); + let projection_exec = + Arc::new(ProjectionExec::try_new(proj_exprs, filter_child)?) as _; + + Ok(FilterPushdownResult { + support: FilterPushdownSupport::Supported { + child_descriptions, + op: projection_exec, + retry: false, + }, + remaining_description, + }) + } else { + Ok(FilterPushdownResult { + support: FilterPushdownSupport::Supported { + child_descriptions, + op: filter_child, + retry: true, + }, + remaining_description, + }) + } } } From bff47bec767865a1d32cc7d4d69f81daa3bcf8c4 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Wed, 16 Apr 2025 19:45:10 +0300 Subject: [PATCH 31/40] update projection tests --- datafusion/core/tests/dataframe/mod.rs | 106 +++++++++--------- .../physical_optimizer/filter_pushdown.rs | 8 +- 2 files changed, 58 insertions(+), 56 deletions(-) diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index 1855a512048d6..e4f7387afd136 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -2852,32 +2852,33 @@ async fn test_count_wildcard_on_where_scalar_subquery() -> Result<()> { assert_snapshot!( pretty_format_batches(&sql_results).unwrap(), @r" - +---------------+---------------------------------------------------------------------------------------------------------------------------+ - | plan_type | plan | - +---------------+---------------------------------------------------------------------------------------------------------------------------+ - | logical_plan | Projection: t1.a, t1.b | - | | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) | - | | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true | - | | Left Join: t1.a = __scalar_sq_1.a | - | | TableScan: t1 projection=[a, b] | - | | SubqueryAlias: __scalar_sq_1 | - | | Projection: count(Int64(1)) AS count(*), t2.a, Boolean(true) AS __always_true | - | | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1))]] | - | | TableScan: t2 projection=[a] | - | physical_plan | CoalesceBatchesExec: target_batch_size=8192 | - | | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] | - | | CoalesceBatchesExec: target_batch_size=8192 | - | | HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, a@1)], projection=[a@0, b@1, count(*)@2, __always_true@4] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | ProjectionExec: expr=[count(Int64(1))@1 as count(*), a@0 as a, true as __always_true] | - | | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))] | - | | CoalesceBatchesExec: target_batch_size=8192 | - | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 | - | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | - | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | | - +---------------+---------------------------------------------------------------------------------------------------------------------------+ + +---------------+-----------------------------------------------------------------------------------------------------------------------------+ + | plan_type | plan | + +---------------+-----------------------------------------------------------------------------------------------------------------------------+ + | logical_plan | Projection: t1.a, t1.b | + | | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) | + | | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true | + | | Left Join: t1.a = __scalar_sq_1.a | + | | TableScan: t1 projection=[a, b] | + | | SubqueryAlias: __scalar_sq_1 | + | | Projection: count(Int64(1)) AS count(*), t2.a, Boolean(true) AS __always_true | + | | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1))]] | + | | TableScan: t2 projection=[a] | + | physical_plan | CoalesceBatchesExec: target_batch_size=8192 | + | | ProjectionExec: expr=[a@0 as a, b@1 as b] | + | | CoalesceBatchesExec: target_batch_size=8192 | + | | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0 | + | | HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, a@1)], projection=[a@0, b@1, count(*)@2, __always_true@4] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | ProjectionExec: expr=[count(Int64(1))@1 as count(*), a@0 as a, true as __always_true] | + | | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))] | + | | CoalesceBatchesExec: target_batch_size=8192 | + | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 | + | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | + | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | | + +---------------+-----------------------------------------------------------------------------------------------------------------------------+ " ); @@ -2909,32 +2910,33 @@ async fn test_count_wildcard_on_where_scalar_subquery() -> Result<()> { assert_snapshot!( pretty_format_batches(&df_results).unwrap(), @r" - +---------------+---------------------------------------------------------------------------------------------------------------------------+ - | plan_type | plan | - +---------------+---------------------------------------------------------------------------------------------------------------------------+ - | logical_plan | Projection: t1.a, t1.b | - | | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) | - | | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true | - | | Left Join: t1.a = __scalar_sq_1.a | - | | TableScan: t1 projection=[a, b] | - | | SubqueryAlias: __scalar_sq_1 | - | | Projection: count(*), t2.a, Boolean(true) AS __always_true | - | | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1)) AS count(*)]] | - | | TableScan: t2 projection=[a] | - | physical_plan | CoalesceBatchesExec: target_batch_size=8192 | - | | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] | - | | CoalesceBatchesExec: target_batch_size=8192 | - | | HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, a@1)], projection=[a@0, b@1, count(*)@2, __always_true@4] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | ProjectionExec: expr=[count(*)@1 as count(*), a@0 as a, true as __always_true] | - | | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(*)] | - | | CoalesceBatchesExec: target_batch_size=8192 | - | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 | - | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | - | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(*)] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | | - +---------------+---------------------------------------------------------------------------------------------------------------------------+ + +---------------+-----------------------------------------------------------------------------------------------------------------------------+ + | plan_type | plan | + +---------------+-----------------------------------------------------------------------------------------------------------------------------+ + | logical_plan | Projection: t1.a, t1.b | + | | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) | + | | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true | + | | Left Join: t1.a = __scalar_sq_1.a | + | | TableScan: t1 projection=[a, b] | + | | SubqueryAlias: __scalar_sq_1 | + | | Projection: count(*), t2.a, Boolean(true) AS __always_true | + | | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1)) AS count(*)]] | + | | TableScan: t2 projection=[a] | + | physical_plan | CoalesceBatchesExec: target_batch_size=8192 | + | | ProjectionExec: expr=[a@0 as a, b@1 as b] | + | | CoalesceBatchesExec: target_batch_size=8192 | + | | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0 | + | | HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, a@1)], projection=[a@0, b@1, count(*)@2, __always_true@4] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | ProjectionExec: expr=[count(*)@1 as count(*), a@0 as a, true as __always_true] | + | | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(*)] | + | | CoalesceBatchesExec: target_batch_size=8192 | + | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 | + | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | + | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(*)] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | | + +---------------+-----------------------------------------------------------------------------------------------------------------------------+ " ); diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs index da1ea3e506a0b..5a03c68593404 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs @@ -299,8 +299,8 @@ fn test_filter_with_projection() { - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test output: Ok: - - FilterExec: a@1 = foo, projection=[b@1, a@0] - - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, predicate=true + - ProjectionExec: expr=[b@1 as b, a@0 as a] + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, predicate=a@1 = foo ", ); @@ -322,8 +322,8 @@ fn test_filter_with_projection() { - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test output: Ok: - - FilterExec: a@0 = foo, projection=[b@1] - - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, predicate=true + - ProjectionExec: expr=[b@1 as b] + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, predicate=a@0 = foo " ); } From ce49ad45f2c26970731e9046a4620c469843c992 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Wed, 16 Apr 2025 15:27:24 -0500 Subject: [PATCH 32/40] update slt files --- .../sqllogictest/test_files/aggregate.slt | 4 +- datafusion/sqllogictest/test_files/array.slt | 30 +- .../test_files/create_external_table.slt | 33 +- datafusion/sqllogictest/test_files/cte.slt | 22 +- .../sqllogictest/test_files/describe.slt | 18 +- .../sqllogictest/test_files/dynamic_file.slt | 11 +- .../sqllogictest/test_files/explain.slt | 78 +-- .../sqllogictest/test_files/explain_tree.slt | 444 +++++++++--------- .../test_files/filter_without_sort_exec.slt | 24 +- .../sqllogictest/test_files/intersection.slt | 13 +- .../sqllogictest/test_files/join.slt.part | 6 +- .../join_disable_repartition_joins.slt | 4 +- datafusion/sqllogictest/test_files/joins.slt | 24 +- datafusion/sqllogictest/test_files/order.slt | 13 +- .../sqllogictest/test_files/parquet.slt | 49 +- .../test_files/parquet_filter_pushdown.slt | 4 +- .../sqllogictest/test_files/predicates.slt | 63 +-- .../sqllogictest/test_files/repartition.slt | 4 +- datafusion/sqllogictest/test_files/select.slt | 24 +- .../sqllogictest/test_files/subquery.slt | 11 +- datafusion/sqllogictest/test_files/union.slt | 12 +- datafusion/sqllogictest/test_files/window.slt | 16 +- 22 files changed, 372 insertions(+), 535 deletions(-) diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 621e212ebc718..4f8c3f6c7719a 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -5213,8 +5213,8 @@ physical_plan 08)--------------CoalescePartitionsExec 09)----------------AggregateExec: mode=Partial, gby=[c2@0 as c2, c3@1 as c3], aggr=[] 10)------------------CoalesceBatchesExec: target_batch_size=8192 -11)--------------------FilterExec: c3@1 >= 10 AND c3@1 <= 20 -12)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +11)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +12)----------------------FilterExec: c3@1 >= 10 AND c3@1 <= 20 13)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], file_type=csv, has_header=true query I diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index f9bbcedff5eef..fd1d94354c279 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -5992,7 +5992,7 @@ logical_plan 02)--Aggregate: groupBy=[[]], aggr=[[count(Int64(1))]] 03)----SubqueryAlias: test 04)------SubqueryAlias: t -05)--------Projection: +05)--------Projection: 06)----------Filter: substr(CAST(md5(CAST(tmp_table.value AS Utf8)) AS Utf8), Int64(1), Int64(32)) IN ([Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278"), Utf8View("a"), Utf8View("b"), Utf8View("c")]) 07)------------TableScan: tmp_table projection=[value] physical_plan @@ -6002,8 +6002,8 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] 05)--------ProjectionExec: expr=[] 06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8)), 1, 32) IN ([Literal { value: Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278") }, Literal { value: Utf8View("a") }, Literal { value: Utf8View("b") }, Literal { value: Utf8View("c") }]) -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------FilterExec: substr(md5(CAST(value@0 AS Utf8)), 1, 32) IN ([Literal { value: Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278") }, Literal { value: Utf8View("a") }, Literal { value: Utf8View("b") }, Literal { value: Utf8View("c") }]) 09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] query I @@ -6021,7 +6021,7 @@ logical_plan 02)--Aggregate: groupBy=[[]], aggr=[[count(Int64(1))]] 03)----SubqueryAlias: test 04)------SubqueryAlias: t -05)--------Projection: +05)--------Projection: 06)----------Filter: substr(CAST(md5(CAST(tmp_table.value AS Utf8)) AS Utf8), Int64(1), Int64(32)) IN ([Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278"), Utf8View("a"), Utf8View("b"), Utf8View("c")]) 07)------------TableScan: tmp_table projection=[value] physical_plan @@ -6031,8 +6031,8 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] 05)--------ProjectionExec: expr=[] 06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8)), 1, 32) IN ([Literal { value: Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278") }, Literal { value: Utf8View("a") }, Literal { value: Utf8View("b") }, Literal { value: Utf8View("c") }]) -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------FilterExec: substr(md5(CAST(value@0 AS Utf8)), 1, 32) IN ([Literal { value: Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278") }, Literal { value: Utf8View("a") }, Literal { value: Utf8View("b") }, Literal { value: Utf8View("c") }]) 09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] query I @@ -6050,7 +6050,7 @@ logical_plan 02)--Aggregate: groupBy=[[]], aggr=[[count(Int64(1))]] 03)----SubqueryAlias: test 04)------SubqueryAlias: t -05)--------Projection: +05)--------Projection: 06)----------Filter: substr(CAST(md5(CAST(tmp_table.value AS Utf8)) AS Utf8), Int64(1), Int64(32)) IN ([Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278"), Utf8View("a"), Utf8View("b"), Utf8View("c")]) 07)------------TableScan: tmp_table projection=[value] physical_plan @@ -6060,8 +6060,8 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] 05)--------ProjectionExec: expr=[] 06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8)), 1, 32) IN ([Literal { value: Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278") }, Literal { value: Utf8View("a") }, Literal { value: Utf8View("b") }, Literal { value: Utf8View("c") }]) -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------FilterExec: substr(md5(CAST(value@0 AS Utf8)), 1, 32) IN ([Literal { value: Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278") }, Literal { value: Utf8View("a") }, Literal { value: Utf8View("b") }, Literal { value: Utf8View("c") }]) 09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] # FIXME: due to rewrite below not working, this is _extremely_ slow to evaluate @@ -6081,7 +6081,7 @@ logical_plan 02)--Aggregate: groupBy=[[]], aggr=[[count(Int64(1))]] 03)----SubqueryAlias: test 04)------SubqueryAlias: t -05)--------Projection: +05)--------Projection: 06)----------Filter: array_has(LargeList([7f4b18de3cfeb9b4ac78c381ee2ad278, a, b, c]), substr(CAST(md5(CAST(tmp_table.value AS Utf8)) AS Utf8), Int64(1), Int64(32))) 07)------------TableScan: tmp_table projection=[value] physical_plan @@ -6091,8 +6091,8 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] 05)--------ProjectionExec: expr=[] 06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------FilterExec: array_has([7f4b18de3cfeb9b4ac78c381ee2ad278, a, b, c], substr(md5(CAST(value@0 AS Utf8)), 1, 32)) -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------FilterExec: array_has([7f4b18de3cfeb9b4ac78c381ee2ad278, a, b, c], substr(md5(CAST(value@0 AS Utf8)), 1, 32)) 09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] query I @@ -6110,7 +6110,7 @@ logical_plan 02)--Aggregate: groupBy=[[]], aggr=[[count(Int64(1))]] 03)----SubqueryAlias: test 04)------SubqueryAlias: t -05)--------Projection: +05)--------Projection: 06)----------Filter: substr(CAST(md5(CAST(tmp_table.value AS Utf8)) AS Utf8), Int64(1), Int64(32)) IN ([Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278"), Utf8View("a"), Utf8View("b"), Utf8View("c")]) 07)------------TableScan: tmp_table projection=[value] physical_plan @@ -6120,8 +6120,8 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] 05)--------ProjectionExec: expr=[] 06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8)), 1, 32) IN ([Literal { value: Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278") }, Literal { value: Utf8View("a") }, Literal { value: Utf8View("b") }, Literal { value: Utf8View("c") }]) -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------FilterExec: substr(md5(CAST(value@0 AS Utf8)), 1, 32) IN ([Literal { value: Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278") }, Literal { value: Utf8View("a") }, Literal { value: Utf8View("b") }, Literal { value: Utf8View("c") }]) 09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] query I diff --git a/datafusion/sqllogictest/test_files/create_external_table.slt b/datafusion/sqllogictest/test_files/create_external_table.slt index bb66aef2514c9..f64a80d76273f 100644 --- a/datafusion/sqllogictest/test_files/create_external_table.slt +++ b/datafusion/sqllogictest/test_files/create_external_table.slt @@ -243,55 +243,40 @@ OPTIONS ( # Create an external parquet table and infer schema to order by # query should succeed -statement ok +statement error DataFusion error: Object Store error: Object at location /Users/adriangb/GitHub/datafusion/parquet\-testing/data/alltypes_plain\.parquet not found: No such file or directory \(os error 2\) CREATE EXTERNAL TABLE t STORED AS parquet LOCATION '../../parquet-testing/data/alltypes_plain.parquet' WITH ORDER (id); ## Verify that the table is created with a sort order. Explain should show output_ordering=[id@0 ASC] -query TT +query error DataFusion error: Error during planning: table 'datafusion\.public\.t' not found EXPLAIN SELECT id FROM t ORDER BY id ASC; ----- -logical_plan -01)Sort: t.id ASC NULLS LAST -02)--TableScan: t projection=[id] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet ## Test a DESC order and verify that output_ordering is ASC from the previous OBRDER BY -query TT +query error DataFusion error: Error during planning: table 'datafusion\.public\.t' not found EXPLAIN SELECT id FROM t ORDER BY id DESC; ----- -logical_plan -01)Sort: t.id DESC NULLS FIRST -02)--TableScan: t projection=[id] -physical_plan -01)SortExec: expr=[id@0 DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet -statement ok +statement error DataFusion error: Execution error: Table 't' doesn't exist\. DROP TABLE t; # Create table with non default sort order -statement ok +statement error DataFusion error: Object Store error: Object at location /Users/adriangb/GitHub/datafusion/parquet\-testing/data/alltypes_plain\.parquet not found: No such file or directory \(os error 2\) CREATE EXTERNAL TABLE t STORED AS parquet LOCATION '../../parquet-testing/data/alltypes_plain.parquet' WITH ORDER (id DESC NULLS FIRST); ## Verify that the table is created with a sort order. Explain should show output_ordering=[id@0 DESC NULLS FIRST] -query TT +query error DataFusion error: Error during planning: table 'datafusion\.public\.t' not found EXPLAIN SELECT id FROM t; ----- -logical_plan TableScan: t projection=[id] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id], output_ordering=[id@0 DESC], file_type=parquet -statement ok +statement error DataFusion error: Execution error: Table 't' doesn't exist\. DROP TABLE t; # query should fail with bad column -statement error DataFusion error: Error during planning: Column foo is not in schema +statement error DataFusion error: Object Store error: Object at location /Users/adriangb/GitHub/datafusion/parquet\-testing/data/alltypes_plain\.parquet not found: No such file or directory \(os error 2\) CREATE EXTERNAL TABLE t STORED AS parquet LOCATION '../../parquet-testing/data/alltypes_plain.parquet' WITH ORDER (foo); # Create external table with qualified name should belong to the schema statement ok CREATE SCHEMA staging; -statement ok +statement error DataFusion error: Object Store error: Object at location /Users/adriangb/GitHub/datafusion/parquet\-testing/data/alltypes_plain\.parquet not found: No such file or directory \(os error 2\) CREATE EXTERNAL TABLE staging.foo STORED AS parquet LOCATION '../../parquet-testing/data/alltypes_plain.parquet'; # Create external table with qualified name, but no schema should error diff --git a/datafusion/sqllogictest/test_files/cte.slt b/datafusion/sqllogictest/test_files/cte.slt index e019af9775a42..20930ec6a4c58 100644 --- a/datafusion/sqllogictest/test_files/cte.slt +++ b/datafusion/sqllogictest/test_files/cte.slt @@ -118,8 +118,8 @@ physical_plan 04)--CoalescePartitionsExec 05)----ProjectionExec: expr=[id@0 + 1 as id] 06)------CoalesceBatchesExec: target_batch_size=8192 -07)--------FilterExec: id@0 < 10 -08)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +07)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)----------FilterExec: id@0 < 10 09)------------WorkTableExec: name=nodes # setup @@ -164,8 +164,8 @@ physical_plan 04)----CoalescePartitionsExec 05)------ProjectionExec: expr=[time@0 + 1 as time, name@1 as name, account_balance@2 + 10 as account_balance] 06)--------CoalesceBatchesExec: target_batch_size=2 -07)----------FilterExec: time@0 < 10 -08)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +07)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)------------FilterExec: time@0 < 10 09)--------------WorkTableExec: name=balances # recursive CTE with static term derived from table works @@ -722,7 +722,7 @@ logical_plan 03)----Projection: Int64(1) AS val 04)------EmptyRelation 05)----Projection: Int64(2) AS val -06)------Cross Join: +06)------Cross Join: 07)--------Filter: recursive_cte.val < Int64(2) 08)----------TableScan: recursive_cte 09)--------SubqueryAlias: sub_cte @@ -736,8 +736,8 @@ physical_plan 05)----CrossJoinExec 06)------CoalescePartitionsExec 07)--------CoalesceBatchesExec: target_batch_size=8182 -08)----------FilterExec: val@0 < 2 -09)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)------------FilterExec: val@0 < 2 10)--------------WorkTableExec: name=recursive_cte 11)------ProjectionExec: expr=[2 as val] 12)--------PlaceholderRowExec @@ -966,8 +966,8 @@ physical_plan 04)--CoalescePartitionsExec 05)----ProjectionExec: expr=[n@0 + 1 as numbers.n + Int64(1)] 06)------CoalesceBatchesExec: target_batch_size=8182 -07)--------FilterExec: n@0 < 10 -08)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +07)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)----------FilterExec: n@0 < 10 09)------------WorkTableExec: name=numbers query TT @@ -992,8 +992,8 @@ physical_plan 04)--CoalescePartitionsExec 05)----ProjectionExec: expr=[n@0 + 1 as numbers.n + Int64(1)] 06)------CoalesceBatchesExec: target_batch_size=8182 -07)--------FilterExec: n@0 < 10 -08)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +07)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)----------FilterExec: n@0 < 10 09)------------WorkTableExec: name=numbers statement count 0 diff --git a/datafusion/sqllogictest/test_files/describe.slt b/datafusion/sqllogictest/test_files/describe.slt index e4cb30628eec5..6317bdf4ca0b7 100644 --- a/datafusion/sqllogictest/test_files/describe.slt +++ b/datafusion/sqllogictest/test_files/describe.slt @@ -67,22 +67,8 @@ DESCRIBE '../core/tests/data/aggregate_simple.csv'; # Describe command ########## -statement ok +statement error DataFusion error: Object Store error: Object at location /Users/adriangb/GitHub/datafusion/parquet\-testing/data/alltypes_tiny_pages\.parquet not found: No such file or directory \(os error 2\) CREATE EXTERNAL TABLE alltypes_tiny_pages STORED AS PARQUET LOCATION '../../parquet-testing/data/alltypes_tiny_pages.parquet'; -query TTT +query error DataFusion error: Error during planning: table 'datafusion\.public\.alltypes_tiny_pages' not found describe alltypes_tiny_pages; ----- -id Int32 YES -bool_col Boolean YES -tinyint_col Int8 YES -smallint_col Int16 YES -int_col Int32 YES -bigint_col Int64 YES -float_col Float32 YES -double_col Float64 YES -date_string_col Utf8View YES -string_col Utf8View YES -timestamp_col Timestamp(Nanosecond, None) YES -year Int32 YES -month Int32 YES diff --git a/datafusion/sqllogictest/test_files/dynamic_file.slt b/datafusion/sqllogictest/test_files/dynamic_file.slt index 69f9a43ad4077..2f7147f1250c7 100644 --- a/datafusion/sqllogictest/test_files/dynamic_file.slt +++ b/datafusion/sqllogictest/test_files/dynamic_file.slt @@ -254,14 +254,5 @@ SELECT a, b FROM '../core/tests/data/2.json' 5 -3.5 7 -3.5 -query IT +query error DataFusion error: Error during planning: table 'datafusion\.public\.\.\./\.\./parquet\-testing/data/alltypes_plain\.parquet' not found SELECT id, CAST(string_col AS varchar) FROM '../../parquet-testing/data/alltypes_plain.parquet'; ----- -4 0 -5 1 -6 0 -7 1 -2 0 -3 1 -0 0 -1 1 diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index ba2596551f1d5..e87637d4d3da8 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -44,8 +44,8 @@ logical_plan 03)----TableScan: aggregate_test_100 projection=[c1, c2], partial_filters=[aggregate_test_100.c2 > Int8(10)] physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: c2@1 > 10, projection=[c1@0] -03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +03)----FilterExec: c2@1 > 10, projection=[c1@0] 04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2], file_type=csv, has_header=true # explain_csv_exec_scan_config @@ -279,89 +279,23 @@ physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/ statement ok set datafusion.execution.collect_statistics = true; -statement ok +statement error DataFusion error: Object Store error: Object at location /Users/adriangb/GitHub/datafusion/parquet\-testing/data/alltypes_plain\.parquet not found: No such file or directory \(os error 2\) CREATE EXTERNAL TABLE alltypes_plain STORED AS PARQUET LOCATION '../../parquet-testing/data/alltypes_plain.parquet'; -query TT +query error DataFusion error: Error during planning: table 'datafusion\.public\.alltypes_plain' not found EXPLAIN SELECT * FROM alltypes_plain limit 10; ----- -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] # explain verbose with both collect & show statistics on -query TT +query error DataFusion error: Error during planning: table 'datafusion\.public\.alltypes_plain' not found EXPLAIN VERBOSE SELECT * FROM alltypes_plain limit 10; ----- -initial_physical_plan -01)GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] -initial_physical_plan_with_schema -01)GlobalLimitExec: skip=0, fetch=10, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] -physical_plan after OutputRequirements -01)OutputRequirementExec, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] -02)--GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] -physical_plan after aggregate_statistics SAME TEXT AS ABOVE -physical_plan after join_selection SAME TEXT AS ABOVE -physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE -physical_plan after EnforceDistribution SAME TEXT AS ABOVE -physical_plan after CombinePartialFinalAggregate SAME TEXT AS ABOVE -physical_plan after EnforceSorting SAME TEXT AS ABOVE -physical_plan after OptimizeAggregateOrder SAME TEXT AS ABOVE -physical_plan after ProjectionPushdown SAME TEXT AS ABOVE -physical_plan after coalesce_batches SAME TEXT AS ABOVE -physical_plan after OutputRequirements -01)GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] -physical_plan after LimitAggregation SAME TEXT AS ABOVE -physical_plan after PushdownFilter SAME TEXT AS ABOVE -physical_plan after LimitPushdown DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] -physical_plan after ProjectionPushdown SAME TEXT AS ABOVE -physical_plan after SanityCheckPlan SAME TEXT AS ABOVE -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] -physical_plan_with_schema DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] statement ok set datafusion.explain.show_statistics = false; # explain verbose with collect on and & show statistics off: still has stats -query TT +query error DataFusion error: Error during planning: table 'datafusion\.public\.alltypes_plain' not found EXPLAIN VERBOSE SELECT * FROM alltypes_plain limit 10; ----- -initial_physical_plan -01)GlobalLimitExec: skip=0, fetch=10 -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet -initial_physical_plan_with_stats -01)GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] -initial_physical_plan_with_schema -01)GlobalLimitExec: skip=0, fetch=10, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] -physical_plan after OutputRequirements -01)OutputRequirementExec -02)--GlobalLimitExec: skip=0, fetch=10 -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet -physical_plan after aggregate_statistics SAME TEXT AS ABOVE -physical_plan after join_selection SAME TEXT AS ABOVE -physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE -physical_plan after EnforceDistribution SAME TEXT AS ABOVE -physical_plan after CombinePartialFinalAggregate SAME TEXT AS ABOVE -physical_plan after EnforceSorting SAME TEXT AS ABOVE -physical_plan after OptimizeAggregateOrder SAME TEXT AS ABOVE -physical_plan after ProjectionPushdown SAME TEXT AS ABOVE -physical_plan after coalesce_batches SAME TEXT AS ABOVE -physical_plan after OutputRequirements -01)GlobalLimitExec: skip=0, fetch=10 -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet -physical_plan after LimitAggregation SAME TEXT AS ABOVE -physical_plan after PushdownFilter SAME TEXT AS ABOVE -physical_plan after LimitPushdown DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet -physical_plan after ProjectionPushdown SAME TEXT AS ABOVE -physical_plan after SanityCheckPlan SAME TEXT AS ABOVE -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet -physical_plan_with_stats DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] -physical_plan_with_schema DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] statement ok diff --git a/datafusion/sqllogictest/test_files/explain_tree.slt b/datafusion/sqllogictest/test_files/explain_tree.slt index 7a0e322eb8bcd..a9b374385e62c 100644 --- a/datafusion/sqllogictest/test_files/explain_tree.slt +++ b/datafusion/sqllogictest/test_files/explain_tree.slt @@ -172,19 +172,19 @@ physical_plan 05)│ 8192 │ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ FilterExec │ +08)│ RepartitionExec │ 09)│ -------------------- │ -10)│ predicate: │ -11)│ string_col != foo │ -12)└─────────────┬─────────────┘ -13)┌─────────────┴─────────────┐ -14)│ RepartitionExec │ -15)│ -------------------- │ -16)│ output_partition_count: │ -17)│ 1 │ -18)│ │ -19)│ partitioning_scheme: │ -20)│ RoundRobinBatch(4) │ +10)│ output_partition_count: │ +11)│ 1 │ +12)│ │ +13)│ partitioning_scheme: │ +14)│ RoundRobinBatch(4) │ +15)└─────────────┬─────────────┘ +16)┌─────────────┴─────────────┐ +17)│ FilterExec │ +18)│ -------------------- │ +19)│ predicate: │ +20)│ string_col != foo │ 21)└─────────────┬─────────────┘ 22)┌─────────────┴─────────────┐ 23)│ DataSourceExec │ @@ -422,23 +422,23 @@ physical_plan 05)│ 8192 │ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ FilterExec │ +08)│ RepartitionExec │ 09)│ -------------------- │ -10)│ predicate: │ -11)│ string_col != foo AND │ -12)│ string_col != bar │ -13)│ AND string_col != a │ -14)│ really long string │ -15)│ constant │ -16)└─────────────┬─────────────┘ -17)┌─────────────┴─────────────┐ -18)│ RepartitionExec │ -19)│ -------------------- │ -20)│ output_partition_count: │ -21)│ 1 │ -22)│ │ -23)│ partitioning_scheme: │ -24)│ RoundRobinBatch(4) │ +10)│ output_partition_count: │ +11)│ 1 │ +12)│ │ +13)│ partitioning_scheme: │ +14)│ RoundRobinBatch(4) │ +15)└─────────────┬─────────────┘ +16)┌─────────────┴─────────────┐ +17)│ FilterExec │ +18)│ -------------------- │ +19)│ predicate: │ +20)│ string_col != foo AND │ +21)│ string_col != bar │ +22)│ AND string_col != a │ +23)│ really long string │ +24)│ constant │ 25)└─────────────┬─────────────┘ 26)┌─────────────┴─────────────┐ 27)│ DataSourceExec │ @@ -460,20 +460,20 @@ physical_plan 05)│ 8192 │ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ FilterExec │ +08)│ RepartitionExec │ 09)│ -------------------- │ -10)│ predicate: │ -11)│ string_col != │ -12)│ aaaaaaaaaaaa │ -13)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ -14)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ -15)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ -16)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ -17)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ -18)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ -19)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ -20)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ -21)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +10)│ output_partition_count: │ +11)│ 1 │ +12)│ │ +13)│ partitioning_scheme: │ +14)│ RoundRobinBatch(4) │ +15)└─────────────┬─────────────┘ +16)┌─────────────┴─────────────┐ +17)│ FilterExec │ +18)│ -------------------- │ +19)│ predicate: │ +20)│ string_col != │ +21)│ aaaaaaaaaaaa │ 22)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ 23)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ 24)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ @@ -491,16 +491,16 @@ physical_plan 36)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ 37)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ 38)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ -39)│ ... │ -40)└─────────────┬─────────────┘ -41)┌─────────────┴─────────────┐ -42)│ RepartitionExec │ -43)│ -------------------- │ -44)│ output_partition_count: │ -45)│ 1 │ -46)│ │ -47)│ partitioning_scheme: │ -48)│ RoundRobinBatch(4) │ +39)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +40)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +41)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +42)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +43)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +44)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +45)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +46)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +47)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +48)│ ... │ 49)└─────────────┬─────────────┘ 50)┌─────────────┴─────────────┐ 51)│ DataSourceExec │ @@ -522,19 +522,19 @@ physical_plan 05)│ 8192 │ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ FilterExec │ +08)│ RepartitionExec │ 09)│ -------------------- │ -10)│ predicate: │ -11)│string_col != aaaaaaaaaaaaa│ -12)└─────────────┬─────────────┘ -13)┌─────────────┴─────────────┐ -14)│ RepartitionExec │ -15)│ -------------------- │ -16)│ output_partition_count: │ -17)│ 1 │ -18)│ │ -19)│ partitioning_scheme: │ -20)│ RoundRobinBatch(4) │ +10)│ output_partition_count: │ +11)│ 1 │ +12)│ │ +13)│ partitioning_scheme: │ +14)│ RoundRobinBatch(4) │ +15)└─────────────┬─────────────┘ +16)┌─────────────┴─────────────┐ +17)│ FilterExec │ +18)│ -------------------- │ +19)│ predicate: │ +20)│string_col != aaaaaaaaaaaaa│ 21)└─────────────┬─────────────┘ 22)┌─────────────┴─────────────┐ 23)│ DataSourceExec │ @@ -556,21 +556,21 @@ physical_plan 05)│ 8192 │ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ FilterExec │ +08)│ RepartitionExec │ 09)│ -------------------- │ -10)│ predicate: │ -11)│ string_col != │ -12)│ aaaaaaaaaaaa │ -13)│ aaa │ -14)└─────────────┬─────────────┘ -15)┌─────────────┴─────────────┐ -16)│ RepartitionExec │ -17)│ -------------------- │ -18)│ output_partition_count: │ -19)│ 1 │ -20)│ │ -21)│ partitioning_scheme: │ -22)│ RoundRobinBatch(4) │ +10)│ output_partition_count: │ +11)│ 1 │ +12)│ │ +13)│ partitioning_scheme: │ +14)│ RoundRobinBatch(4) │ +15)└─────────────┬─────────────┘ +16)┌─────────────┴─────────────┐ +17)│ FilterExec │ +18)│ -------------------- │ +19)│ predicate: │ +20)│ string_col != │ +21)│ aaaaaaaaaaaa │ +22)│ aaa │ 23)└─────────────┬─────────────┘ 24)┌─────────────┴─────────────┐ 25)│ DataSourceExec │ @@ -591,19 +591,19 @@ physical_plan 05)│ 8192 │ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ FilterExec │ +08)│ RepartitionExec │ 09)│ -------------------- │ -10)│ predicate: │ -11)│ string_col != foo │ -12)└─────────────┬─────────────┘ -13)┌─────────────┴─────────────┐ -14)│ RepartitionExec │ -15)│ -------------------- │ -16)│ output_partition_count: │ -17)│ 1 │ -18)│ │ -19)│ partitioning_scheme: │ -20)│ RoundRobinBatch(4) │ +10)│ output_partition_count: │ +11)│ 1 │ +12)│ │ +13)│ partitioning_scheme: │ +14)│ RoundRobinBatch(4) │ +15)└─────────────┬─────────────┘ +16)┌─────────────┴─────────────┐ +17)│ FilterExec │ +18)│ -------------------- │ +19)│ predicate: │ +20)│ string_col != foo │ 21)└─────────────┬─────────────┘ 22)┌─────────────┴─────────────┐ 23)│ DataSourceExec │ @@ -625,19 +625,19 @@ physical_plan 05)│ 8192 │ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ FilterExec │ +08)│ RepartitionExec │ 09)│ -------------------- │ -10)│ predicate: │ -11)│ string_col != foo │ -12)└─────────────┬─────────────┘ -13)┌─────────────┴─────────────┐ -14)│ RepartitionExec │ -15)│ -------------------- │ -16)│ output_partition_count: │ -17)│ 1 │ -18)│ │ -19)│ partitioning_scheme: │ -20)│ RoundRobinBatch(4) │ +10)│ output_partition_count: │ +11)│ 1 │ +12)│ │ +13)│ partitioning_scheme: │ +14)│ RoundRobinBatch(4) │ +15)└─────────────┬─────────────┘ +16)┌─────────────┴─────────────┐ +17)│ FilterExec │ +18)│ -------------------- │ +19)│ predicate: │ +20)│ string_col != foo │ 21)└─────────────┬─────────────┘ 22)┌─────────────┴─────────────┐ 23)│ DataSourceExec │ @@ -686,19 +686,19 @@ physical_plan 05)│ 8192 │ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ FilterExec │ +08)│ RepartitionExec │ 09)│ -------------------- │ -10)│ predicate: │ -11)│ string_col != foo │ -12)└─────────────┬─────────────┘ -13)┌─────────────┴─────────────┐ -14)│ RepartitionExec │ -15)│ -------------------- │ -16)│ output_partition_count: │ -17)│ 1 │ -18)│ │ -19)│ partitioning_scheme: │ -20)│ RoundRobinBatch(4) │ +10)│ output_partition_count: │ +11)│ 1 │ +12)│ │ +13)│ partitioning_scheme: │ +14)│ RoundRobinBatch(4) │ +15)└─────────────┬─────────────┘ +16)┌─────────────┴─────────────┐ +17)│ FilterExec │ +18)│ -------------------- │ +19)│ predicate: │ +20)│ string_col != foo │ 21)└─────────────┬─────────────┘ 22)┌─────────────┴─────────────┐ 23)│ DataSourceExec │ @@ -719,19 +719,19 @@ physical_plan 05)│ 8192 │ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ FilterExec │ +08)│ RepartitionExec │ 09)│ -------------------- │ -10)│ predicate: │ -11)│ string_col != foo │ -12)└─────────────┬─────────────┘ -13)┌─────────────┴─────────────┐ -14)│ RepartitionExec │ -15)│ -------------------- │ -16)│ output_partition_count: │ -17)│ 1 │ -18)│ │ -19)│ partitioning_scheme: │ -20)│ RoundRobinBatch(4) │ +10)│ output_partition_count: │ +11)│ 1 │ +12)│ │ +13)│ partitioning_scheme: │ +14)│ RoundRobinBatch(4) │ +15)└─────────────┬─────────────┘ +16)┌─────────────┴─────────────┐ +17)│ FilterExec │ +18)│ -------------------- │ +19)│ predicate: │ +20)│ string_col != foo │ 21)└─────────────┬─────────────┘ 22)┌─────────────┴─────────────┐ 23)│ DataSourceExec │ @@ -1599,18 +1599,18 @@ physical_plan 11)│ 8192 │ 12)└─────────────┬─────────────┘ 13)┌─────────────┴─────────────┐ -14)│ FilterExec │ +14)│ RepartitionExec │ 15)│ -------------------- │ -16)│ predicate: ticker = A │ -17)└─────────────┬─────────────┘ -18)┌─────────────┴─────────────┐ -19)│ RepartitionExec │ -20)│ -------------------- │ -21)│ output_partition_count: │ -22)│ 1 │ -23)│ │ -24)│ partitioning_scheme: │ -25)│ RoundRobinBatch(4) │ +16)│ output_partition_count: │ +17)│ 1 │ +18)│ │ +19)│ partitioning_scheme: │ +20)│ RoundRobinBatch(4) │ +21)└─────────────┬─────────────┘ +22)┌─────────────┴─────────────┐ +23)│ FilterExec │ +24)│ -------------------- │ +25)│ predicate: ticker = A │ 26)└─────────────┬─────────────┘ 27)┌─────────────┴─────────────┐ 28)│ StreamingTableExec │ @@ -1639,20 +1639,20 @@ physical_plan 10)│ 8192 │ 11)└─────────────┬─────────────┘ 12)┌─────────────┴─────────────┐ -13)│ FilterExec │ +13)│ RepartitionExec │ 14)│ -------------------- │ -15)│ predicate: │ -16)│ ticker = A AND CAST(time │ -17)│ AS Date32) = date │ -18)└─────────────┬─────────────┘ -19)┌─────────────┴─────────────┐ -20)│ RepartitionExec │ -21)│ -------------------- │ -22)│ output_partition_count: │ -23)│ 1 │ -24)│ │ -25)│ partitioning_scheme: │ -26)│ RoundRobinBatch(4) │ +15)│ output_partition_count: │ +16)│ 1 │ +17)│ │ +18)│ partitioning_scheme: │ +19)│ RoundRobinBatch(4) │ +20)└─────────────┬─────────────┘ +21)┌─────────────┴─────────────┐ +22)│ FilterExec │ +23)│ -------------------- │ +24)│ predicate: │ +25)│ ticker = A AND CAST(time │ +26)│ AS Date32) = date │ 27)└─────────────┬─────────────┘ 28)┌─────────────┴─────────────┐ 29)│ StreamingTableExec │ @@ -1680,20 +1680,20 @@ physical_plan 10)│ 8192 │ 11)└─────────────┬─────────────┘ 12)┌─────────────┴─────────────┐ -13)│ FilterExec │ +13)│ RepartitionExec │ 14)│ -------------------- │ -15)│ predicate: │ -16)│ ticker = A AND CAST(time │ -17)│ AS Date32) = date │ -18)└─────────────┬─────────────┘ -19)┌─────────────┴─────────────┐ -20)│ RepartitionExec │ -21)│ -------------------- │ -22)│ output_partition_count: │ -23)│ 1 │ -24)│ │ -25)│ partitioning_scheme: │ -26)│ RoundRobinBatch(4) │ +15)│ output_partition_count: │ +16)│ 1 │ +17)│ │ +18)│ partitioning_scheme: │ +19)│ RoundRobinBatch(4) │ +20)└─────────────┬─────────────┘ +21)┌─────────────┴─────────────┐ +22)│ FilterExec │ +23)│ -------------------- │ +24)│ predicate: │ +25)│ ticker = A AND CAST(time │ +26)│ AS Date32) = date │ 27)└─────────────┬─────────────┘ 28)┌─────────────┴─────────────┐ 29)│ StreamingTableExec │ @@ -1719,20 +1719,20 @@ physical_plan 08)│ 8192 │ 09)└─────────────┬─────────────┘ 10)┌─────────────┴─────────────┐ -11)│ FilterExec │ +11)│ RepartitionExec │ 12)│ -------------------- │ -13)│ predicate: │ -14)│ ticker = A AND CAST(time │ -15)│ AS Date32) = date │ -16)└─────────────┬─────────────┘ -17)┌─────────────┴─────────────┐ -18)│ RepartitionExec │ -19)│ -------------------- │ -20)│ output_partition_count: │ -21)│ 1 │ -22)│ │ -23)│ partitioning_scheme: │ -24)│ RoundRobinBatch(4) │ +13)│ output_partition_count: │ +14)│ 1 │ +15)│ │ +16)│ partitioning_scheme: │ +17)│ RoundRobinBatch(4) │ +18)└─────────────┬─────────────┘ +19)┌─────────────┴─────────────┐ +20)│ FilterExec │ +21)│ -------------------- │ +22)│ predicate: │ +23)│ ticker = A AND CAST(time │ +24)│ AS Date32) = date │ 25)└─────────────┬─────────────┘ 26)┌─────────────┴─────────────┐ 27)│ StreamingTableExec │ @@ -1762,20 +1762,20 @@ physical_plan 11)│ 8192 │ 12)└─────────────┬─────────────┘ 13)┌─────────────┴─────────────┐ -14)│ FilterExec │ +14)│ RepartitionExec │ 15)│ -------------------- │ -16)│ predicate: │ -17)│ ticker = A AND CAST(time │ -18)│ AS Date32) = date │ -19)└─────────────┬─────────────┘ -20)┌─────────────┴─────────────┐ -21)│ RepartitionExec │ -22)│ -------------------- │ -23)│ output_partition_count: │ -24)│ 1 │ -25)│ │ -26)│ partitioning_scheme: │ -27)│ RoundRobinBatch(4) │ +16)│ output_partition_count: │ +17)│ 1 │ +18)│ │ +19)│ partitioning_scheme: │ +20)│ RoundRobinBatch(4) │ +21)└─────────────┬─────────────┘ +22)┌─────────────┴─────────────┐ +23)│ FilterExec │ +24)│ -------------------- │ +25)│ predicate: │ +26)│ ticker = A AND CAST(time │ +27)│ AS Date32) = date │ 28)└─────────────┬─────────────┘ 29)┌─────────────┴─────────────┐ 30)│ StreamingTableExec │ @@ -1807,19 +1807,19 @@ physical_plan 11)│ 8192 │ 12)└─────────────┬─────────────┘ 13)┌─────────────┴─────────────┐ -14)│ FilterExec │ +14)│ RepartitionExec │ 15)│ -------------------- │ -16)│ predicate: │ -17)│ date = 2006-01-02 │ -18)└─────────────┬─────────────┘ -19)┌─────────────┴─────────────┐ -20)│ RepartitionExec │ -21)│ -------------------- │ -22)│ output_partition_count: │ -23)│ 1 │ -24)│ │ -25)│ partitioning_scheme: │ -26)│ RoundRobinBatch(4) │ +16)│ output_partition_count: │ +17)│ 1 │ +18)│ │ +19)│ partitioning_scheme: │ +20)│ RoundRobinBatch(4) │ +21)└─────────────┬─────────────┘ +22)┌─────────────┴─────────────┐ +23)│ FilterExec │ +24)│ -------------------- │ +25)│ predicate: │ +26)│ date = 2006-01-02 │ 27)└─────────────┬─────────────┘ 28)┌─────────────┴─────────────┐ 29)│ StreamingTableExec │ @@ -1862,18 +1862,18 @@ physical_plan 18)-----------------------------│ 8192 │ 19)-----------------------------└─────────────┬─────────────┘ 20)-----------------------------┌─────────────┴─────────────┐ -21)-----------------------------│ FilterExec │ +21)-----------------------------│ RepartitionExec │ 22)-----------------------------│ -------------------- │ -23)-----------------------------│ predicate: id < 10 │ -24)-----------------------------└─────────────┬─────────────┘ -25)-----------------------------┌─────────────┴─────────────┐ -26)-----------------------------│ RepartitionExec │ -27)-----------------------------│ -------------------- │ -28)-----------------------------│ output_partition_count: │ -29)-----------------------------│ 1 │ -30)-----------------------------│ │ -31)-----------------------------│ partitioning_scheme: │ -32)-----------------------------│ RoundRobinBatch(4) │ +23)-----------------------------│ output_partition_count: │ +24)-----------------------------│ 1 │ +25)-----------------------------│ │ +26)-----------------------------│ partitioning_scheme: │ +27)-----------------------------│ RoundRobinBatch(4) │ +28)-----------------------------└─────────────┬─────────────┘ +29)-----------------------------┌─────────────┴─────────────┐ +30)-----------------------------│ FilterExec │ +31)-----------------------------│ -------------------- │ +32)-----------------------------│ predicate: id < 10 │ 33)-----------------------------└─────────────┬─────────────┘ 34)-----------------------------┌─────────────┴─────────────┐ 35)-----------------------------│ WorkTableExec │ @@ -2055,18 +2055,18 @@ physical_plan 12)│ 8192 │ 13)└─────────────┬─────────────┘ 14)┌─────────────┴─────────────┐ -15)│ FilterExec │ +15)│ RepartitionExec │ 16)│ -------------------- │ -17)│ predicate: c3 > 0 │ -18)└─────────────┬─────────────┘ -19)┌─────────────┴─────────────┐ -20)│ RepartitionExec │ -21)│ -------------------- │ -22)│ output_partition_count: │ -23)│ 1 │ -24)│ │ -25)│ partitioning_scheme: │ -26)│ RoundRobinBatch(4) │ +17)│ output_partition_count: │ +18)│ 1 │ +19)│ │ +20)│ partitioning_scheme: │ +21)│ RoundRobinBatch(4) │ +22)└─────────────┬─────────────┘ +23)┌─────────────┴─────────────┐ +24)│ FilterExec │ +25)│ -------------------- │ +26)│ predicate: c3 > 0 │ 27)└─────────────┬─────────────┘ 28)┌─────────────┴─────────────┐ 29)│ StreamingTableExec │ diff --git a/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt b/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt index d96044fda8c05..c892ae17d26c6 100644 --- a/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt +++ b/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt @@ -39,8 +39,8 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [date@0 ASC NULLS LAST, time@2 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: ticker@1 = A -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------FilterExec: ticker@1 = A 05)--------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] # constant ticker, CAST(time AS DATE) = time, order by time @@ -56,8 +56,8 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [time@2 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 05)--------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] # same thing but order by date @@ -73,8 +73,8 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [date@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 05)--------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] # same thing but order by ticker @@ -90,8 +90,8 @@ logical_plan physical_plan 01)CoalescePartitionsExec 02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 05)--------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] # same thing but order by time, date @@ -107,8 +107,8 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [time@2 ASC NULLS LAST, date@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 05)--------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] # CAST(time AS DATE) <> date (should require a sort) @@ -148,6 +148,6 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: date@0 = 2006-01-02 -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------FilterExec: date@0 = 2006-01-02 05)--------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] diff --git a/datafusion/sqllogictest/test_files/intersection.slt b/datafusion/sqllogictest/test_files/intersection.slt index 301878cc98e2c..f7aaf2dee5a3b 100644 --- a/datafusion/sqllogictest/test_files/intersection.slt +++ b/datafusion/sqllogictest/test_files/intersection.slt @@ -15,7 +15,7 @@ # specific language governing permissions and limitations # under the License. -statement ok +statement error DataFusion error: Object Store error: Object at location /Users/adriangb/GitHub/datafusion/parquet\-testing/data/alltypes_plain\.parquet not found: No such file or directory \(os error 2\) CREATE EXTERNAL TABLE alltypes_plain STORED AS PARQUET LOCATION '../../parquet-testing/data/alltypes_plain.parquet'; query ?I @@ -30,15 +30,8 @@ SELECT * FROM (SELECT null AS id1, 1 AS id2) t1 ---- NULL 1 -query IR +query error DataFusion error: Error during planning: table 'datafusion\.public\.alltypes_plain' not found SELECT int_col, double_col FROM alltypes_plain where int_col > 0 INTERSECT ALL SELECT int_col, double_col FROM alltypes_plain LIMIT 4 ----- -1 10.1 -1 10.1 -1 10.1 -1 10.1 -query IR +query error DataFusion error: Error during planning: table 'datafusion\.public\.alltypes_plain' not found SELECT int_col, double_col FROM alltypes_plain where int_col > 0 INTERSECT SELECT int_col, double_col FROM alltypes_plain ----- -1 10.1 diff --git a/datafusion/sqllogictest/test_files/join.slt.part b/datafusion/sqllogictest/test_files/join.slt.part index 972dd2265343d..f4228d1b011f5 100644 --- a/datafusion/sqllogictest/test_files/join.slt.part +++ b/datafusion/sqllogictest/test_files/join.slt.part @@ -984,9 +984,9 @@ logical_plan 08)--------TableScan: department projection=[emp_id, dept_name] physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: dept_name@2 != Engineering AND name@1 = Alice OR name@1 != Alice AND name@1 = Carol -03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)------CoalesceBatchesExec: target_batch_size=8192 +02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +03)----CoalesceBatchesExec: target_batch_size=8192 +04)------FilterExec: dept_name@2 != Engineering AND name@1 = Alice OR name@1 != Alice AND name@1 = Carol 05)--------HashJoinExec: mode=CollectLeft, join_type=Left, on=[(emp_id@0, emp_id@0)], projection=[emp_id@0, name@1, dept_name@3] 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------FilterExec: name@1 = Alice OR name@1 != Alice AND name@1 = Carol diff --git a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt index a1efc1317b4aa..898d35937d9e4 100644 --- a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt +++ b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt @@ -100,8 +100,8 @@ physical_plan 04)------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d@1, d@3), (c@0, c@2)], projection=[a@0, b@1] 05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], file_type=csv, has_header=true 06)--------CoalesceBatchesExec: target_batch_size=8192 -07)----------FilterExec: d@3 = 3 -08)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +07)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)------------FilterExec: d@3 = 3 09)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], file_type=csv, has_header=true # preserve_right_semi_join diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index ca86dbfcc3c16..0b676183bc2a0 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -2053,12 +2053,12 @@ physical_plan 02)--NestedLoopJoinExec: join_type=Inner, filter=t1_id@0 > t2_id@1 03)----CoalescePartitionsExec 04)------CoalesceBatchesExec: target_batch_size=2 -05)--------FilterExec: t2_int@1 > 1, projection=[t2_id@0] -06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------FilterExec: t2_int@1 > 1, projection=[t2_id@0] 07)------------DataSourceExec: partitions=1, partition_sizes=[1] 08)----CoalesceBatchesExec: target_batch_size=2 -09)------FilterExec: t1_id@0 > 10 -10)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +10)--------FilterExec: t1_id@0 > 10 11)----------DataSourceExec: partitions=1, partition_sizes=[1] query II @@ -2092,12 +2092,12 @@ physical_plan 01)NestedLoopJoinExec: join_type=Right, filter=t1_id@0 < t2_id@1 02)--CoalescePartitionsExec 03)----CoalesceBatchesExec: target_batch_size=2 -04)------FilterExec: t1_id@0 > 22 -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------FilterExec: t1_id@0 > 22 06)----------DataSourceExec: partitions=1, partition_sizes=[1] 07)--CoalesceBatchesExec: target_batch_size=2 -08)----FilterExec: t2_id@0 > 11 -09)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)------FilterExec: t2_id@0 > 11 10)--------DataSourceExec: partitions=1, partition_sizes=[1] query II @@ -4428,8 +4428,8 @@ physical_plan 03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(a@0, a@0)] 04)------CoalescePartitionsExec 05)--------CoalesceBatchesExec: target_batch_size=3 -06)----------FilterExec: b@1 > 3, projection=[a@0] -07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)------------FilterExec: b@1 > 3, projection=[a@0] 08)--------------DataSourceExec: partitions=1, partition_sizes=[1] 09)------SortExec: expr=[c@2 DESC], preserve_partitioning=[true] 10)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -4452,8 +4452,8 @@ physical_plan 03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(a@0, a@0)] 04)------CoalescePartitionsExec 05)--------CoalesceBatchesExec: target_batch_size=3 -06)----------FilterExec: b@1 > 3, projection=[a@0] -07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)------------FilterExec: b@1 > 3, projection=[a@0] 08)--------------DataSourceExec: partitions=1, partition_sizes=[1] 09)------SortExec: expr=[c@2 DESC NULLS LAST], preserve_partitioning=[true] 10)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 diff --git a/datafusion/sqllogictest/test_files/order.slt b/datafusion/sqllogictest/test_files/order.slt index 4e8be56f3377d..bcda014a861ef 100644 --- a/datafusion/sqllogictest/test_files/order.slt +++ b/datafusion/sqllogictest/test_files/order.slt @@ -16,7 +16,7 @@ # under the License. -statement ok +statement error DataFusion error: Object Store error: Object at location /Users/adriangb/GitHub/datafusion/parquet\-testing/data/alltypes_plain\.parquet not found: No such file or directory \(os error 2\) CREATE EXTERNAL TABLE alltypes_plain STORED AS PARQUET LOCATION '../../parquet-testing/data/alltypes_plain.parquet'; statement ok @@ -40,17 +40,8 @@ LOCATION '../../testing/data/csv/aggregate_test_100.csv' OPTIONS ('format.has_header' 'true'); # test_sort_unprojected_col -query I +query error DataFusion error: Error during planning: table 'datafusion\.public\.alltypes_plain' not found SELECT id FROM alltypes_plain ORDER BY int_col, double_col ----- -4 -6 -2 -0 -5 -7 -3 -1 # test_order_by_agg_expr diff --git a/datafusion/sqllogictest/test_files/parquet.slt b/datafusion/sqllogictest/test_files/parquet.slt index 2970b2effb3e9..3848ef3da02a4 100644 --- a/datafusion/sqllogictest/test_files/parquet.slt +++ b/datafusion/sqllogictest/test_files/parquet.slt @@ -190,17 +190,8 @@ STORED AS PARQUET LOCATION '../../parquet-testing/data/alltypes_plain.parquet'; # Test a basic query with a CAST: -query IT +query error DataFusion error: Object Store error: Object at location /Users/adriangb/GitHub/datafusion/parquet\-testing/data/alltypes_plain\.parquet not found: No such file or directory \(os error 2\) SELECT id, CAST(string_col AS varchar) FROM alltypes_plain ----- -4 0 -5 1 -6 0 -7 1 -2 0 -3 1 -0 0 -1 1 # Ensure that local files can not be read by default (a potential security issue) # (url table is only supported when DynamicFileCatalog is enabled) @@ -309,40 +300,32 @@ statement ok DROP TABLE timestamp_with_tz; # Test a query from the single_nan data set: -statement ok +statement error DataFusion error: Object Store error: Object at location /Users/adriangb/GitHub/datafusion/parquet\-testing/data/single_nan\.parquet not found: No such file or directory \(os error 2\) CREATE EXTERNAL TABLE single_nan STORED AS PARQUET LOCATION '../../parquet-testing/data/single_nan.parquet'; # Check table size: -query I +query error DataFusion error: Error during planning: table 'datafusion\.public\.single_nan' not found SELECT COUNT(*) FROM single_nan; ----- -1 # Query for the single NULL: -query R +query error DataFusion error: Error during planning: table 'datafusion\.public\.single_nan' not found SELECT mycol FROM single_nan; ----- -NULL # Clean up -statement ok +statement error DataFusion error: Execution error: Table 'single_nan' doesn't exist\. DROP TABLE single_nan; -statement ok +statement error DataFusion error: Object Store error: Object at location /Users/adriangb/GitHub/datafusion/parquet\-testing/data/list_columns\.parquet not found: No such file or directory \(os error 2\) CREATE EXTERNAL TABLE list_columns STORED AS PARQUET LOCATION '../../parquet-testing/data/list_columns.parquet'; -query ?? +query error DataFusion error: Error during planning: table 'datafusion\.public\.list_columns' not found SELECT int64_list, utf8_list FROM list_columns ----- -[1, 2, 3] [abc, efg, hij] -[NULL, 1] NULL -[4] [efg, NULL, hij, xyz] -statement ok +statement error DataFusion error: Execution error: Table 'list_columns' doesn't exist\. DROP TABLE list_columns; # Clean up @@ -409,8 +392,8 @@ logical_plan 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%")] physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 -02)--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% -03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +03)----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% 04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=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% @@ -457,8 +440,8 @@ logical_plan 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%")] physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% -03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +03)----FilterExec: binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% 04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% @@ -508,8 +491,8 @@ logical_plan 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%")] physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% -03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +03)----FilterExec: binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% 04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% @@ -623,8 +606,8 @@ logical_plan 02)--TableScan: foo projection=[column1], partial_filters=[foo.column1 LIKE Utf8View("f%")] physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: column1@0 LIKE f% -03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +03)----FilterExec: column1@0 LIKE f% 04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/foo.parquet]]}, projection=[column1], file_type=parquet, predicate=column1@0 LIKE f%, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= g AND f <= column1_max@1, required_guarantees=[] statement ok diff --git a/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt b/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt index 758113b708355..709376805a216 100644 --- a/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt +++ b/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt @@ -111,8 +111,8 @@ physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----CoalesceBatchesExec: target_batch_size=8192 -04)------FilterExec: b@1 > 2, projection=[a@0] -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 +05)--------FilterExec: b@1 > 2, projection=[a@0] 06)----------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, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] # also test querying on columns that are not in all the files diff --git a/datafusion/sqllogictest/test_files/predicates.slt b/datafusion/sqllogictest/test_files/predicates.slt index b263e39f3b11b..e09bd4b0eeb3c 100644 --- a/datafusion/sqllogictest/test_files/predicates.slt +++ b/datafusion/sqllogictest/test_files/predicates.slt @@ -42,7 +42,7 @@ STORED AS CSV LOCATION '../../testing/data/csv/aggregate_test_100.csv' OPTIONS ('format.has_header' 'true'); -statement ok +statement error DataFusion error: Object Store error: Object at location /Users/adriangb/GitHub/datafusion/parquet\-testing/data/alltypes_plain\.parquet not found: No such file or directory \(os error 2\) CREATE EXTERNAL TABLE alltypes_plain STORED AS PARQUET LOCATION '../../parquet-testing/data/alltypes_plain.parquet'; @@ -438,19 +438,12 @@ set datafusion.explain.logical_plan_only = false # async fn test_expect_all -query IR +query error DataFusion error: Error during planning: table 'datafusion\.public\.alltypes_plain' not found SELECT int_col, double_col FROM alltypes_plain where int_col > 0 EXCEPT ALL SELECT int_col, double_col FROM alltypes_plain where int_col < 1 ----- -1 10.1 -1 10.1 -1 10.1 -1 10.1 # async fn test_expect_distinct -query IR +query error DataFusion error: Error during planning: table 'datafusion\.public\.alltypes_plain' not found SELECT int_col, double_col FROM alltypes_plain where int_col > 0 EXCEPT SELECT int_col, double_col FROM alltypes_plain where int_col < 1 ----- -1 10.1 ######## @@ -460,7 +453,7 @@ SELECT int_col, double_col FROM alltypes_plain where int_col > 0 EXCEPT SELECT i statement ok drop table aggregate_test_100; -statement ok +statement error DataFusion error: Execution error: Table 'alltypes_plain' doesn't exist\. drop table alltypes_plain; statement ok @@ -514,7 +507,7 @@ DROP TABLE t; # Refer to https://github.com/apache/datafusion/pull/7821#pullrequestreview-1688062599 ######## -statement ok +statement error DataFusion error: Object Store error: Object at location /Users/adriangb/GitHub/datafusion/parquet\-testing/data/data_index_bloom_encoding_stats\.parquet not found: No such file or directory \(os error 2\) CREATE EXTERNAL TABLE data_index_bloom_encoding_stats STORED AS PARQUET LOCATION '../../parquet-testing/data/data_index_bloom_encoding_stats.parquet'; query TT @@ -522,24 +515,14 @@ SHOW datafusion.execution.parquet.bloom_filter_on_read ---- datafusion.execution.parquet.bloom_filter_on_read true -query T +query error DataFusion error: Error during planning: table 'datafusion\.public\.data_index_bloom_encoding_stats' not found SELECT * FROM data_index_bloom_encoding_stats WHERE "String" = 'foo'; ----- -query T +query error DataFusion error: Error during planning: table 'datafusion\.public\.data_index_bloom_encoding_stats' not found SELECT * FROM data_index_bloom_encoding_stats WHERE "String" = 'test'; ----- -test -query T +query error DataFusion error: Error during planning: table 'datafusion\.public\.data_index_bloom_encoding_stats' not found SELECT * FROM data_index_bloom_encoding_stats WHERE "String" like '%e%'; ----- -Hello -test -are you -the quick -over -the lazy ######## @@ -550,24 +533,14 @@ the lazy statement ok set datafusion.execution.parquet.bloom_filter_on_read=false; -query T +query error DataFusion error: Error during planning: table 'datafusion\.public\.data_index_bloom_encoding_stats' not found SELECT * FROM data_index_bloom_encoding_stats WHERE "String" = 'foo'; ----- -query T +query error DataFusion error: Error during planning: table 'datafusion\.public\.data_index_bloom_encoding_stats' not found SELECT * FROM data_index_bloom_encoding_stats WHERE "String" = 'test'; ----- -test -query T +query error DataFusion error: Error during planning: table 'datafusion\.public\.data_index_bloom_encoding_stats' not found SELECT * FROM data_index_bloom_encoding_stats WHERE "String" like '%e%'; ----- -Hello -test -are you -the quick -over -the lazy statement ok set datafusion.execution.parquet.bloom_filter_on_read=true; @@ -576,7 +549,7 @@ set datafusion.execution.parquet.bloom_filter_on_read=true; ######## # Clean up after the test ######## -statement ok +statement error DataFusion error: Execution error: Table 'data_index_bloom_encoding_stats' doesn't exist\. DROP TABLE data_index_bloom_encoding_stats; @@ -673,14 +646,14 @@ physical_plan 03)----CoalesceBatchesExec: target_batch_size=8192 04)------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------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 -07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +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 08)--------------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 09)----CoalesceBatchesExec: target_batch_size=8192 10)------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 11)--------CoalesceBatchesExec: target_batch_size=8192 -12)----------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 -13)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +12)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +13)------------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 14)--------------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 ######## @@ -773,8 +746,8 @@ physical_plan 12)------------CoalesceBatchesExec: target_batch_size=8192 13)--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 14)----------------CoalesceBatchesExec: target_batch_size=8192 -15)------------------FilterExec: p_brand@1 = Brand#12 OR p_brand@1 = Brand#23, projection=[p_partkey@0] -16)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +15)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +16)--------------------FilterExec: p_brand@1 = Brand#12 OR p_brand@1 = Brand#23, projection=[p_partkey@0] 17)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand], file_type=csv, has_header=true # Inlist simplification diff --git a/datafusion/sqllogictest/test_files/repartition.slt b/datafusion/sqllogictest/test_files/repartition.slt index 70666346e2cab..7041430155561 100644 --- a/datafusion/sqllogictest/test_files/repartition.slt +++ b/datafusion/sqllogictest/test_files/repartition.slt @@ -123,8 +123,8 @@ logical_plan physical_plan 01)CoalescePartitionsExec: fetch=5 02)--CoalesceBatchesExec: target_batch_size=8192, fetch=5 -03)----FilterExec: c3@2 > 0 -04)------RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 +03)----RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 +04)------FilterExec: c3@2 > 0 05)--------StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true # Start repratition on empty column test. diff --git a/datafusion/sqllogictest/test_files/select.slt b/datafusion/sqllogictest/test_files/select.slt index aa14faf984e40..e16b81651c36f 100644 --- a/datafusion/sqllogictest/test_files/select.slt +++ b/datafusion/sqllogictest/test_files/select.slt @@ -1441,8 +1441,8 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: a@1 = 0 -04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +04)------FilterExec: a@1 = 0 05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true # source is ordered by a,b,c @@ -1462,8 +1462,8 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [c@3 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: a@1 = 0 AND b@2 = 0 -04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +04)------FilterExec: a@1 = 0 AND b@2 = 0 05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true # source is ordered by a,b,c @@ -1483,8 +1483,8 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: a@1 = 0 AND b@2 = 0 -04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +04)------FilterExec: a@1 = 0 AND b@2 = 0 05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true # source is ordered by a,b,c @@ -1504,8 +1504,8 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: a@1 = 0 AND b@2 = 0 -04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +04)------FilterExec: a@1 = 0 AND b@2 = 0 05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true # source is ordered by a,b,c @@ -1526,8 +1526,8 @@ physical_plan 01)SortPreservingMergeExec: [c@3 ASC NULLS LAST] 02)--SortExec: expr=[c@3 ASC NULLS LAST], preserve_partitioning=[true] 03)----CoalesceBatchesExec: target_batch_size=8192 -04)------FilterExec: a@1 = 0 OR b@2 = 0 -05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +05)--------FilterExec: a@1 = 0 OR b@2 = 0 06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true # When ordering lost during projection, we shouldn't keep the SortExec. @@ -1574,8 +1574,8 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [CAST(round(CAST(b@2 AS Float64)) AS Int32) ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: CAST(round(CAST(b@2 AS Float64)) AS Int32) = a@1 -04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +04)------FilterExec: CAST(round(CAST(b@2 AS Float64)) AS Int32) = a@1 05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index aaccaaa43ce49..0d91471c55424 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -1150,12 +1150,13 @@ logical_plan 06)--------TableScan: t2 projection=[t2_id] physical_plan 01)CoalesceBatchesExec: target_batch_size=2 -02)--FilterExec: t1_id@0 > 40 OR NOT mark@3, projection=[t1_id@0, t1_name@1, t1_int@2] +02)--ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int] 03)----CoalesceBatchesExec: target_batch_size=2 -04)------HashJoinExec: mode=CollectLeft, join_type=LeftMark, on=[(t1_id@0, t2_id@0)] -05)--------DataSourceExec: partitions=1, partition_sizes=[1] -06)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)----------DataSourceExec: partitions=1, partition_sizes=[1] +04)------FilterExec: t1_id@0 > 40 OR NOT mark@3 +05)--------HashJoinExec: mode=CollectLeft, join_type=LeftMark, on=[(t1_id@0, t2_id@0)] +06)----------DataSourceExec: partitions=1, partition_sizes=[1] +07)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)------------DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.explain.logical_plan_only = true; diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index 356f1598bc0fa..cf286c8402358 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -516,8 +516,8 @@ physical_plan 10)------------------RepartitionExec: partitioning=Hash([c1@0], 4), input_partitions=4 11)--------------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[] 12)----------------------CoalesceBatchesExec: target_batch_size=2 -13)------------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434, projection=[c1@0] -14)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +13)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +14)--------------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434, projection=[c1@0] 15)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], file_type=csv, has_header=true 16)----ProjectionExec: expr=[1 as cnt] 17)------PlaceholderRowExec @@ -837,12 +837,12 @@ physical_plan 01)CoalescePartitionsExec 02)--UnionExec 03)----CoalesceBatchesExec: target_batch_size=2 -04)------FilterExec: c1@0 = a -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------FilterExec: c1@0 = a 06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], file_type=csv, has_header=true 07)----CoalesceBatchesExec: target_batch_size=2 -08)------FilterExec: c1@0 = a -09)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)--------FilterExec: c1@0 = a 10)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], file_type=csv, has_header=true # Clean up after the test diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 76e3751e4b8e4..449929adf037f 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -1783,8 +1783,8 @@ physical_plan 08)--------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 09)----------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[] 10)------------------CoalesceBatchesExec: target_batch_size=4096 -11)--------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434, projection=[c1@0] -12)----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +11)--------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +12)----------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434, projection=[c1@0] 13)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], file_type=csv, has_header=true @@ -5205,8 +5205,8 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=1 06)----------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 07)------------CoalesceBatchesExec: target_batch_size=1 -08)--------------FilterExec: c1@0 = 2 OR c1@0 = 3 -09)----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +09)----------------FilterExec: c1@0 = 2 OR c1@0 = 3 10)------------------DataSourceExec: partitions=1, partition_sizes=[1] query III @@ -5291,8 +5291,8 @@ physical_plan 07)------------CoalesceBatchesExec: target_batch_size=1 08)--------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 09)----------------CoalesceBatchesExec: target_batch_size=1 -10)------------------FilterExec: c1@0 = 1 -11)--------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +10)------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +11)--------------------FilterExec: c1@0 = 1 12)----------------------DataSourceExec: partitions=1, partition_sizes=[1] query III @@ -5379,8 +5379,8 @@ physical_plan 10)------------------CoalesceBatchesExec: target_batch_size=1 11)--------------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 12)----------------------CoalesceBatchesExec: target_batch_size=1 -13)------------------------FilterExec: c1@0 > 1 -14)--------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +13)------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +14)--------------------------FilterExec: c1@0 > 1 15)----------------------------DataSourceExec: partitions=1, partition_sizes=[1] query IIII From 834f33eae6a97924f5215312d6ddceb4076c4313 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Wed, 16 Apr 2025 15:35:24 -0500 Subject: [PATCH 33/40] fix --- .../test_files/create_external_table.slt | 33 ++++++--- .../sqllogictest/test_files/describe.slt | 18 ++++- .../sqllogictest/test_files/dynamic_file.slt | 11 ++- .../sqllogictest/test_files/explain.slt | 74 ++++++++++++++++++- .../sqllogictest/test_files/intersection.slt | 13 +++- datafusion/sqllogictest/test_files/order.slt | 13 +++- .../sqllogictest/test_files/parquet.slt | 33 +++++++-- .../sqllogictest/test_files/predicates.slt | 51 ++++++++++--- 8 files changed, 205 insertions(+), 41 deletions(-) diff --git a/datafusion/sqllogictest/test_files/create_external_table.slt b/datafusion/sqllogictest/test_files/create_external_table.slt index f64a80d76273f..bb66aef2514c9 100644 --- a/datafusion/sqllogictest/test_files/create_external_table.slt +++ b/datafusion/sqllogictest/test_files/create_external_table.slt @@ -243,40 +243,55 @@ OPTIONS ( # Create an external parquet table and infer schema to order by # query should succeed -statement error DataFusion error: Object Store error: Object at location /Users/adriangb/GitHub/datafusion/parquet\-testing/data/alltypes_plain\.parquet not found: No such file or directory \(os error 2\) +statement ok CREATE EXTERNAL TABLE t STORED AS parquet LOCATION '../../parquet-testing/data/alltypes_plain.parquet' WITH ORDER (id); ## Verify that the table is created with a sort order. Explain should show output_ordering=[id@0 ASC] -query error DataFusion error: Error during planning: table 'datafusion\.public\.t' not found +query TT EXPLAIN SELECT id FROM t ORDER BY id ASC; +---- +logical_plan +01)Sort: t.id ASC NULLS LAST +02)--TableScan: t projection=[id] +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet ## Test a DESC order and verify that output_ordering is ASC from the previous OBRDER BY -query error DataFusion error: Error during planning: table 'datafusion\.public\.t' not found +query TT EXPLAIN SELECT id FROM t ORDER BY id DESC; +---- +logical_plan +01)Sort: t.id DESC NULLS FIRST +02)--TableScan: t projection=[id] +physical_plan +01)SortExec: expr=[id@0 DESC], preserve_partitioning=[false] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet -statement error DataFusion error: Execution error: Table 't' doesn't exist\. +statement ok DROP TABLE t; # Create table with non default sort order -statement error DataFusion error: Object Store error: Object at location /Users/adriangb/GitHub/datafusion/parquet\-testing/data/alltypes_plain\.parquet not found: No such file or directory \(os error 2\) +statement ok CREATE EXTERNAL TABLE t STORED AS parquet LOCATION '../../parquet-testing/data/alltypes_plain.parquet' WITH ORDER (id DESC NULLS FIRST); ## Verify that the table is created with a sort order. Explain should show output_ordering=[id@0 DESC NULLS FIRST] -query error DataFusion error: Error during planning: table 'datafusion\.public\.t' not found +query TT EXPLAIN SELECT id FROM t; +---- +logical_plan TableScan: t projection=[id] +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id], output_ordering=[id@0 DESC], file_type=parquet -statement error DataFusion error: Execution error: Table 't' doesn't exist\. +statement ok DROP TABLE t; # query should fail with bad column -statement error DataFusion error: Object Store error: Object at location /Users/adriangb/GitHub/datafusion/parquet\-testing/data/alltypes_plain\.parquet not found: No such file or directory \(os error 2\) +statement error DataFusion error: Error during planning: Column foo is not in schema CREATE EXTERNAL TABLE t STORED AS parquet LOCATION '../../parquet-testing/data/alltypes_plain.parquet' WITH ORDER (foo); # Create external table with qualified name should belong to the schema statement ok CREATE SCHEMA staging; -statement error DataFusion error: Object Store error: Object at location /Users/adriangb/GitHub/datafusion/parquet\-testing/data/alltypes_plain\.parquet not found: No such file or directory \(os error 2\) +statement ok CREATE EXTERNAL TABLE staging.foo STORED AS parquet LOCATION '../../parquet-testing/data/alltypes_plain.parquet'; # Create external table with qualified name, but no schema should error diff --git a/datafusion/sqllogictest/test_files/describe.slt b/datafusion/sqllogictest/test_files/describe.slt index 6317bdf4ca0b7..e4cb30628eec5 100644 --- a/datafusion/sqllogictest/test_files/describe.slt +++ b/datafusion/sqllogictest/test_files/describe.slt @@ -67,8 +67,22 @@ DESCRIBE '../core/tests/data/aggregate_simple.csv'; # Describe command ########## -statement error DataFusion error: Object Store error: Object at location /Users/adriangb/GitHub/datafusion/parquet\-testing/data/alltypes_tiny_pages\.parquet not found: No such file or directory \(os error 2\) +statement ok CREATE EXTERNAL TABLE alltypes_tiny_pages STORED AS PARQUET LOCATION '../../parquet-testing/data/alltypes_tiny_pages.parquet'; -query error DataFusion error: Error during planning: table 'datafusion\.public\.alltypes_tiny_pages' not found +query TTT describe alltypes_tiny_pages; +---- +id Int32 YES +bool_col Boolean YES +tinyint_col Int8 YES +smallint_col Int16 YES +int_col Int32 YES +bigint_col Int64 YES +float_col Float32 YES +double_col Float64 YES +date_string_col Utf8View YES +string_col Utf8View YES +timestamp_col Timestamp(Nanosecond, None) YES +year Int32 YES +month Int32 YES diff --git a/datafusion/sqllogictest/test_files/dynamic_file.slt b/datafusion/sqllogictest/test_files/dynamic_file.slt index 2f7147f1250c7..69f9a43ad4077 100644 --- a/datafusion/sqllogictest/test_files/dynamic_file.slt +++ b/datafusion/sqllogictest/test_files/dynamic_file.slt @@ -254,5 +254,14 @@ SELECT a, b FROM '../core/tests/data/2.json' 5 -3.5 7 -3.5 -query error DataFusion error: Error during planning: table 'datafusion\.public\.\.\./\.\./parquet\-testing/data/alltypes_plain\.parquet' not found +query IT SELECT id, CAST(string_col AS varchar) FROM '../../parquet-testing/data/alltypes_plain.parquet'; +---- +4 0 +5 1 +6 0 +7 1 +2 0 +3 1 +0 0 +1 1 diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index e87637d4d3da8..8d0753090c4d9 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -279,23 +279,89 @@ physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/ statement ok set datafusion.execution.collect_statistics = true; -statement error DataFusion error: Object Store error: Object at location /Users/adriangb/GitHub/datafusion/parquet\-testing/data/alltypes_plain\.parquet not found: No such file or directory \(os error 2\) +statement ok CREATE EXTERNAL TABLE alltypes_plain STORED AS PARQUET LOCATION '../../parquet-testing/data/alltypes_plain.parquet'; -query error DataFusion error: Error during planning: table 'datafusion\.public\.alltypes_plain' not found +query TT EXPLAIN SELECT * FROM alltypes_plain limit 10; +---- +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] # explain verbose with both collect & show statistics on -query error DataFusion error: Error during planning: table 'datafusion\.public\.alltypes_plain' not found +query TT EXPLAIN VERBOSE SELECT * FROM alltypes_plain limit 10; +---- +initial_physical_plan +01)GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +initial_physical_plan_with_schema +01)GlobalLimitExec: skip=0, fetch=10, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] +physical_plan after OutputRequirements +01)OutputRequirementExec, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +02)--GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +physical_plan after aggregate_statistics SAME TEXT AS ABOVE +physical_plan after join_selection SAME TEXT AS ABOVE +physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE +physical_plan after EnforceDistribution SAME TEXT AS ABOVE +physical_plan after CombinePartialFinalAggregate SAME TEXT AS ABOVE +physical_plan after EnforceSorting SAME TEXT AS ABOVE +physical_plan after OptimizeAggregateOrder SAME TEXT AS ABOVE +physical_plan after ProjectionPushdown SAME TEXT AS ABOVE +physical_plan after coalesce_batches SAME TEXT AS ABOVE +physical_plan after OutputRequirements +01)GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +physical_plan after LimitAggregation SAME TEXT AS ABOVE +physical_plan after PushdownFilter SAME TEXT AS ABOVE +physical_plan after LimitPushdown DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +physical_plan after ProjectionPushdown SAME TEXT AS ABOVE +physical_plan after SanityCheckPlan SAME TEXT AS ABOVE +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +physical_plan_with_schema DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] statement ok set datafusion.explain.show_statistics = false; # explain verbose with collect on and & show statistics off: still has stats -query error DataFusion error: Error during planning: table 'datafusion\.public\.alltypes_plain' not found +query TT EXPLAIN VERBOSE SELECT * FROM alltypes_plain limit 10; +---- +initial_physical_plan +01)GlobalLimitExec: skip=0, fetch=10 +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet +initial_physical_plan_with_stats +01)GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +initial_physical_plan_with_schema +01)GlobalLimitExec: skip=0, fetch=10, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] +physical_plan after OutputRequirements +01)OutputRequirementExec +02)--GlobalLimitExec: skip=0, fetch=10 +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet +physical_plan after aggregate_statistics SAME TEXT AS ABOVE +physical_plan after join_selection SAME TEXT AS ABOVE +physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE +physical_plan after EnforceDistribution SAME TEXT AS ABOVE +physical_plan after CombinePartialFinalAggregate SAME TEXT AS ABOVE +physical_plan after EnforceSorting SAME TEXT AS ABOVE +physical_plan after OptimizeAggregateOrder SAME TEXT AS ABOVE +physical_plan after ProjectionPushdown SAME TEXT AS ABOVE +physical_plan after coalesce_batches SAME TEXT AS ABOVE +physical_plan after OutputRequirements +01)GlobalLimitExec: skip=0, fetch=10 +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet +physical_plan after LimitAggregation SAME TEXT AS ABOVE +physical_plan after PushdownFilter SAME TEXT AS ABOVE +physical_plan after LimitPushdown DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet +physical_plan after ProjectionPushdown SAME TEXT AS ABOVE +physical_plan after SanityCheckPlan SAME TEXT AS ABOVE +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet +physical_plan_with_stats DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +physical_plan_with_schema DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] statement ok diff --git a/datafusion/sqllogictest/test_files/intersection.slt b/datafusion/sqllogictest/test_files/intersection.slt index f7aaf2dee5a3b..301878cc98e2c 100644 --- a/datafusion/sqllogictest/test_files/intersection.slt +++ b/datafusion/sqllogictest/test_files/intersection.slt @@ -15,7 +15,7 @@ # specific language governing permissions and limitations # under the License. -statement error DataFusion error: Object Store error: Object at location /Users/adriangb/GitHub/datafusion/parquet\-testing/data/alltypes_plain\.parquet not found: No such file or directory \(os error 2\) +statement ok CREATE EXTERNAL TABLE alltypes_plain STORED AS PARQUET LOCATION '../../parquet-testing/data/alltypes_plain.parquet'; query ?I @@ -30,8 +30,15 @@ SELECT * FROM (SELECT null AS id1, 1 AS id2) t1 ---- NULL 1 -query error DataFusion error: Error during planning: table 'datafusion\.public\.alltypes_plain' not found +query IR SELECT int_col, double_col FROM alltypes_plain where int_col > 0 INTERSECT ALL SELECT int_col, double_col FROM alltypes_plain LIMIT 4 +---- +1 10.1 +1 10.1 +1 10.1 +1 10.1 -query error DataFusion error: Error during planning: table 'datafusion\.public\.alltypes_plain' not found +query IR SELECT int_col, double_col FROM alltypes_plain where int_col > 0 INTERSECT SELECT int_col, double_col FROM alltypes_plain +---- +1 10.1 diff --git a/datafusion/sqllogictest/test_files/order.slt b/datafusion/sqllogictest/test_files/order.slt index bcda014a861ef..4e8be56f3377d 100644 --- a/datafusion/sqllogictest/test_files/order.slt +++ b/datafusion/sqllogictest/test_files/order.slt @@ -16,7 +16,7 @@ # under the License. -statement error DataFusion error: Object Store error: Object at location /Users/adriangb/GitHub/datafusion/parquet\-testing/data/alltypes_plain\.parquet not found: No such file or directory \(os error 2\) +statement ok CREATE EXTERNAL TABLE alltypes_plain STORED AS PARQUET LOCATION '../../parquet-testing/data/alltypes_plain.parquet'; statement ok @@ -40,8 +40,17 @@ LOCATION '../../testing/data/csv/aggregate_test_100.csv' OPTIONS ('format.has_header' 'true'); # test_sort_unprojected_col -query error DataFusion error: Error during planning: table 'datafusion\.public\.alltypes_plain' not found +query I SELECT id FROM alltypes_plain ORDER BY int_col, double_col +---- +4 +6 +2 +0 +5 +7 +3 +1 # test_order_by_agg_expr diff --git a/datafusion/sqllogictest/test_files/parquet.slt b/datafusion/sqllogictest/test_files/parquet.slt index 3848ef3da02a4..2d274125e0c1e 100644 --- a/datafusion/sqllogictest/test_files/parquet.slt +++ b/datafusion/sqllogictest/test_files/parquet.slt @@ -190,8 +190,17 @@ STORED AS PARQUET LOCATION '../../parquet-testing/data/alltypes_plain.parquet'; # Test a basic query with a CAST: -query error DataFusion error: Object Store error: Object at location /Users/adriangb/GitHub/datafusion/parquet\-testing/data/alltypes_plain\.parquet not found: No such file or directory \(os error 2\) +query IT SELECT id, CAST(string_col AS varchar) FROM alltypes_plain +---- +4 0 +5 1 +6 0 +7 1 +2 0 +3 1 +0 0 +1 1 # Ensure that local files can not be read by default (a potential security issue) # (url table is only supported when DynamicFileCatalog is enabled) @@ -300,32 +309,40 @@ statement ok DROP TABLE timestamp_with_tz; # Test a query from the single_nan data set: -statement error DataFusion error: Object Store error: Object at location /Users/adriangb/GitHub/datafusion/parquet\-testing/data/single_nan\.parquet not found: No such file or directory \(os error 2\) +statement ok CREATE EXTERNAL TABLE single_nan STORED AS PARQUET LOCATION '../../parquet-testing/data/single_nan.parquet'; # Check table size: -query error DataFusion error: Error during planning: table 'datafusion\.public\.single_nan' not found +query I SELECT COUNT(*) FROM single_nan; +---- +1 # Query for the single NULL: -query error DataFusion error: Error during planning: table 'datafusion\.public\.single_nan' not found +query R SELECT mycol FROM single_nan; +---- +NULL # Clean up -statement error DataFusion error: Execution error: Table 'single_nan' doesn't exist\. +statement ok DROP TABLE single_nan; -statement error DataFusion error: Object Store error: Object at location /Users/adriangb/GitHub/datafusion/parquet\-testing/data/list_columns\.parquet not found: No such file or directory \(os error 2\) +statement ok CREATE EXTERNAL TABLE list_columns STORED AS PARQUET LOCATION '../../parquet-testing/data/list_columns.parquet'; -query error DataFusion error: Error during planning: table 'datafusion\.public\.list_columns' not found +query ?? SELECT int64_list, utf8_list FROM list_columns +---- +[1, 2, 3] [abc, efg, hij] +[NULL, 1] NULL +[4] [efg, NULL, hij, xyz] -statement error DataFusion error: Execution error: Table 'list_columns' doesn't exist\. +statement ok DROP TABLE list_columns; # Clean up diff --git a/datafusion/sqllogictest/test_files/predicates.slt b/datafusion/sqllogictest/test_files/predicates.slt index e09bd4b0eeb3c..070c99f67a733 100644 --- a/datafusion/sqllogictest/test_files/predicates.slt +++ b/datafusion/sqllogictest/test_files/predicates.slt @@ -42,7 +42,7 @@ STORED AS CSV LOCATION '../../testing/data/csv/aggregate_test_100.csv' OPTIONS ('format.has_header' 'true'); -statement error DataFusion error: Object Store error: Object at location /Users/adriangb/GitHub/datafusion/parquet\-testing/data/alltypes_plain\.parquet not found: No such file or directory \(os error 2\) +statement ok CREATE EXTERNAL TABLE alltypes_plain STORED AS PARQUET LOCATION '../../parquet-testing/data/alltypes_plain.parquet'; @@ -438,12 +438,19 @@ set datafusion.explain.logical_plan_only = false # async fn test_expect_all -query error DataFusion error: Error during planning: table 'datafusion\.public\.alltypes_plain' not found +query IR SELECT int_col, double_col FROM alltypes_plain where int_col > 0 EXCEPT ALL SELECT int_col, double_col FROM alltypes_plain where int_col < 1 +---- +1 10.1 +1 10.1 +1 10.1 +1 10.1 # async fn test_expect_distinct -query error DataFusion error: Error during planning: table 'datafusion\.public\.alltypes_plain' not found +query IR SELECT int_col, double_col FROM alltypes_plain where int_col > 0 EXCEPT SELECT int_col, double_col FROM alltypes_plain where int_col < 1 +---- +1 10.1 ######## @@ -453,7 +460,7 @@ SELECT int_col, double_col FROM alltypes_plain where int_col > 0 EXCEPT SELECT i statement ok drop table aggregate_test_100; -statement error DataFusion error: Execution error: Table 'alltypes_plain' doesn't exist\. +statement ok drop table alltypes_plain; statement ok @@ -507,7 +514,7 @@ DROP TABLE t; # Refer to https://github.com/apache/datafusion/pull/7821#pullrequestreview-1688062599 ######## -statement error DataFusion error: Object Store error: Object at location /Users/adriangb/GitHub/datafusion/parquet\-testing/data/data_index_bloom_encoding_stats\.parquet not found: No such file or directory \(os error 2\) +statement ok CREATE EXTERNAL TABLE data_index_bloom_encoding_stats STORED AS PARQUET LOCATION '../../parquet-testing/data/data_index_bloom_encoding_stats.parquet'; query TT @@ -515,14 +522,24 @@ SHOW datafusion.execution.parquet.bloom_filter_on_read ---- datafusion.execution.parquet.bloom_filter_on_read true -query error DataFusion error: Error during planning: table 'datafusion\.public\.data_index_bloom_encoding_stats' not found +query T SELECT * FROM data_index_bloom_encoding_stats WHERE "String" = 'foo'; +---- -query error DataFusion error: Error during planning: table 'datafusion\.public\.data_index_bloom_encoding_stats' not found +query T SELECT * FROM data_index_bloom_encoding_stats WHERE "String" = 'test'; +---- +test -query error DataFusion error: Error during planning: table 'datafusion\.public\.data_index_bloom_encoding_stats' not found +query T SELECT * FROM data_index_bloom_encoding_stats WHERE "String" like '%e%'; +---- +Hello +test +are you +the quick +over +the lazy ######## @@ -533,14 +550,24 @@ SELECT * FROM data_index_bloom_encoding_stats WHERE "String" like '%e%'; statement ok set datafusion.execution.parquet.bloom_filter_on_read=false; -query error DataFusion error: Error during planning: table 'datafusion\.public\.data_index_bloom_encoding_stats' not found +query T SELECT * FROM data_index_bloom_encoding_stats WHERE "String" = 'foo'; +---- -query error DataFusion error: Error during planning: table 'datafusion\.public\.data_index_bloom_encoding_stats' not found +query T SELECT * FROM data_index_bloom_encoding_stats WHERE "String" = 'test'; +---- +test -query error DataFusion error: Error during planning: table 'datafusion\.public\.data_index_bloom_encoding_stats' not found +query T SELECT * FROM data_index_bloom_encoding_stats WHERE "String" like '%e%'; +---- +Hello +test +are you +the quick +over +the lazy statement ok set datafusion.execution.parquet.bloom_filter_on_read=true; @@ -549,7 +576,7 @@ set datafusion.execution.parquet.bloom_filter_on_read=true; ######## # Clean up after the test ######## -statement error DataFusion error: Execution error: Table 'data_index_bloom_encoding_stats' doesn't exist\. +statement ok DROP TABLE data_index_bloom_encoding_stats; From 9e5924694ed20dbe028c9f51d090d8685764f84b Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Wed, 16 Apr 2025 17:30:44 -0500 Subject: [PATCH 34/40] fix references --- .../physical-optimizer/src/filter_pushdown.rs | 16 +++------------- .../sqllogictest/test_files/simplify_expr.slt | 1 + 2 files changed, 4 insertions(+), 13 deletions(-) diff --git a/datafusion/physical-optimizer/src/filter_pushdown.rs b/datafusion/physical-optimizer/src/filter_pushdown.rs index bfcd8abf36e25..63999f3579c61 100644 --- a/datafusion/physical-optimizer/src/filter_pushdown.rs +++ b/datafusion/physical-optimizer/src/filter_pushdown.rs @@ -66,17 +66,6 @@ use datafusion_physical_plan::ExecutionPlan; /// amounts of data being read from the source (the projection is `*` so all /// matching columns are read). /// -/// In this simple case we: -/// 1. Enter the recursion with no filters. -/// 2. We find the [`FilterExec`] node and call [`ExecutionPlan::try_pushdown_filters`] on it. -/// 3. The [`FilterExec`] node tries to push it's filters + the filters from the parent nodes (in this case empty) -/// down into it's input, which is the `DataSourceExec` node. -/// 4. The `DataSourceExec` node accepts the filter and returns a [`FilterPushdownResult`] with a new copy of itself -/// and [`FilterPushdown::Exact`] to indicate that the filter was pushed down and the caller no longer -/// needs to handle it. -/// 5. The [`FilterExec`] seeing that all filters were pushed down returns a [`FilterPushdownResult`] that directly -/// returns the new `DataSourceExec` node, effectively removing the [`FilterExec`] node from the plan. -/// /// The new plan looks like: /// /// ```text @@ -367,9 +356,10 @@ use datafusion_physical_plan::ExecutionPlan; /// This is not yet implemented in DataFusion. See /// /// +/// [`PhysicalExpr`]: datafusion_physical_plan::PhysicalExpr /// [`FilterExec`]: datafusion_physical_plan::filter::FilterExec -/// [`ProjectionExec`]: datafusion_phyiscal_plan::projection::ProjectionExec -/// [`AggregateExec`]: datafusion_phyiscal_plan::aggregates::AggregateExec +/// [`ProjectionExec`]: datafusion_physical_plan::projection::ProjectionExec +/// [`AggregateExec`]: datafusion_physical_plan::aggregates::AggregateExec #[derive(Debug)] pub struct PushdownFilter {} diff --git a/datafusion/sqllogictest/test_files/simplify_expr.slt b/datafusion/sqllogictest/test_files/simplify_expr.slt index 075ccafcfd2e0..9985ab49c2da0 100644 --- a/datafusion/sqllogictest/test_files/simplify_expr.slt +++ b/datafusion/sqllogictest/test_files/simplify_expr.slt @@ -107,3 +107,4 @@ query B SELECT a / NULL::DECIMAL(4,3) > 1.2::decimal(2,1) FROM VALUES (1) AS t(a); ---- NULL + From 57a1230ec78d8c03531d13101268c8292fd6103e Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Thu, 17 Apr 2025 13:31:06 +0300 Subject: [PATCH 35/40] improve impls and update tests --- .../physical_optimizer/filter_pushdown.rs | 81 ++++++++++--------- datafusion/datasource/src/file_scan_config.rs | 4 +- datafusion/datasource/src/source.rs | 5 +- .../physical-optimizer/src/filter_pushdown.rs | 36 ++++----- .../physical-plan/src/coalesce_batches.rs | 18 ++--- .../physical-plan/src/execution_plan.rs | 1 - datafusion/physical-plan/src/filter.rs | 34 ++++---- .../physical-plan/src/filter_pushdown.rs | 22 ++++- .../physical-plan/src/repartition/mod.rs | 18 ++--- 9 files changed, 109 insertions(+), 110 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs index 5a03c68593404..5fbba010cc36a 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs @@ -15,6 +15,12 @@ // specific language governing permissions and limitations // under the License. +use std::sync::{Arc, OnceLock}; +use std::{ + any::Any, + fmt::{Display, Formatter}, +}; + use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion::{ datasource::object_store::ObjectStoreUrl, @@ -53,12 +59,8 @@ use datafusion_physical_plan::{ use datafusion_physical_plan::{ displayable, metrics::ExecutionPlanMetricsSet, DisplayFormatType, ExecutionPlan, }; + use object_store::ObjectStore; -use std::sync::{Arc, OnceLock}; -use std::{ - any::Any, - fmt::{Display, Formatter}, -}; /// A placeholder data source that accepts filter pushdown #[derive(Clone, Default)] @@ -130,16 +132,19 @@ impl FileSource for TestSource { fn fmt_extra(&self, t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { match t { DisplayFormatType::Default | DisplayFormatType::Verbose => { + let support = format!(", pushdown_supported={}", self.support); + let predicate_string = self .predicate .as_ref() .map(|p| format!(", predicate={p}")) .unwrap_or_default(); - write!(f, "{}", predicate_string) + write!(f, "{}{}", support, predicate_string) } DisplayFormatType::TreeRender => { if let Some(predicate) = &self.predicate { + writeln!(f, "pushdown_supported={}", fmt_sql(predicate.as_ref()))?; writeln!(f, "predicate={}", fmt_sql(predicate.as_ref()))?; } Ok(()) @@ -159,11 +164,11 @@ impl FileSource for TestSource { op: Arc::new(TestSource { support: true, predicate: Some(conjunction(fd.filters)), - statistics: self.statistics.clone(), // should be updated ? + statistics: self.statistics.clone(), // should be updated in reality }), - retry: false, + revisit: false, }, - remaining_description: Default::default(), + remaining_description: FilterDescription::empty(), }) } else { Ok(filter_pushdown_not_supported(fd)) @@ -191,7 +196,7 @@ fn test_pushdown_into_scan() { // expect the predicate to be pushed down into the DataSource insta::assert_snapshot!( - OptimizationTest::new(plan, PushdownFilter{}), + OptimizationTest::new(plan, PushdownFilter{}, true), @r" OptimizationTest: input: @@ -212,12 +217,11 @@ fn test_pushdown_into_scan_with_config_options() { let plan = Arc::new(FilterExec::try_new(predicate, scan).unwrap()) as _; let mut cfg = ConfigOptions::default(); - cfg.execution.parquet.pushdown_filters = false; insta::assert_snapshot!( - OptimizationTest::new_with_config( + OptimizationTest::new( Arc::clone(&plan), PushdownFilter {}, - &cfg + false ), @r" OptimizationTest: @@ -233,10 +237,10 @@ fn test_pushdown_into_scan_with_config_options() { cfg.execution.parquet.pushdown_filters = true; insta::assert_snapshot!( - OptimizationTest::new_with_config( + OptimizationTest::new( plan, PushdownFilter {}, - &cfg + true ), @r" OptimizationTest: @@ -261,7 +265,7 @@ fn test_filter_collapse() { let plan = Arc::new(FilterExec::try_new(predicate2, filter1).unwrap()); insta::assert_snapshot!( - OptimizationTest::new(plan, PushdownFilter{}), + OptimizationTest::new(plan, PushdownFilter{}, true), @r" OptimizationTest: input: @@ -279,19 +283,17 @@ fn test_filter_collapse() { fn test_filter_with_projection() { let scan = test_scan(true); let projection = vec![1, 0]; - let projected_schema = Arc::new(schema().project(&projection).unwrap()); - let predicate = col_lit_predicate("a", "foo", &projected_schema); + let predicate = col_lit_predicate("a", "foo", schema()); let plan = Arc::new( FilterExec::try_new(predicate, Arc::clone(&scan)) .unwrap() .with_projection(Some(projection)) .unwrap(), ); - // expect the predicate to be pushed down into the DataSource but the FilterExec to be kept for its projection - // the pushed down filters should have their indices adjusted + // expect the predicate to be pushed down into the DataSource but the FilterExec to be converted to ProjectionExec insta::assert_snapshot!( - OptimizationTest::new(plan, PushdownFilter{}), + OptimizationTest::new(plan, PushdownFilter{}, true), @r" OptimizationTest: input: @@ -314,7 +316,7 @@ fn test_filter_with_projection() { .unwrap(), ); insta::assert_snapshot!( - OptimizationTest::new(plan, PushdownFilter{}), + OptimizationTest::new(plan, PushdownFilter{},true), @r" OptimizationTest: input: @@ -343,7 +345,7 @@ fn test_push_down_through_transparent_nodes() { // expect the predicate to be pushed down into the DataSource insta::assert_snapshot!( - OptimizationTest::new(plan, PushdownFilter{}), + OptimizationTest::new(plan, PushdownFilter{},true), @r" OptimizationTest: input: @@ -364,14 +366,16 @@ fn test_push_down_through_transparent_nodes() { #[test] fn test_no_pushdown_through_aggregates() { // There are 2 important points here: - // 1. The outer filter is not pushed down into the aggregate because we haven't - // implemented that yet. + // 1. The outer filter **is not** pushed down at all because we haven't implemented pushdown support + // yet for AggregateExec. // 2. The inner filter **is** pushed down into the DataSource. let scan = test_scan(true); + let filter = Arc::new( FilterExec::try_new(col_lit_predicate("a", "foo", schema()), scan.clone()) .unwrap(), ); + let aggregate_expr = vec![ AggregateExprBuilder::new(count_udaf(), vec![col("a", schema()).unwrap()]) @@ -396,12 +400,15 @@ fn test_no_pushdown_through_aggregates() { ) .unwrap(), ); + + let coalesce = Arc::new(CoalesceBatchesExec::new(aggregate, 100)); + let predicate = col_lit_predicate("a", "foo", schema()); - let plan = Arc::new(FilterExec::try_new(predicate, aggregate).unwrap()); + let plan = Arc::new(FilterExec::try_new(predicate, coalesce).unwrap()); // expect the predicate to be pushed down into the DataSource insta::assert_snapshot!( - OptimizationTest::new(plan, PushdownFilter{}), + OptimizationTest::new(plan, PushdownFilter{}, true), @r" OptimizationTest: input: @@ -459,28 +466,22 @@ pub struct OptimizationTest { } impl OptimizationTest { - pub fn new(input_plan: Arc, opt: O) -> Self - where - O: PhysicalOptimizerRule, - { - let mut parquet_pushdown_config = ConfigOptions::default(); - parquet_pushdown_config.execution.parquet.pushdown_filters = true; - Self::new_with_config(input_plan, opt, &parquet_pushdown_config) - } - - pub fn new_with_config( + pub fn new( input_plan: Arc, opt: O, - config: &ConfigOptions, + allow_pushdown_filters: bool, ) -> Self where O: PhysicalOptimizerRule, { - let input = format_execution_plan(&input_plan); + let mut parquet_pushdown_config = ConfigOptions::default(); + parquet_pushdown_config.execution.parquet.pushdown_filters = + allow_pushdown_filters; + let input = format_execution_plan(&input_plan); let input_schema = input_plan.schema(); - let output_result = opt.optimize(input_plan, config); + let output_result = opt.optimize(input_plan, &parquet_pushdown_config); let output = output_result .and_then(|plan| { if opt.schema_check() && (plan.schema() != input_schema) { diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index 450938e2342ed..4d300470a8976 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -607,7 +607,7 @@ impl DataSource for FileScanConfig { FilterPushdownSupport::Supported { child_descriptions, op, - retry, + revisit: retry, } => { let new_data_source = Arc::new( FileScanConfigBuilder::from(self.clone()) @@ -622,7 +622,7 @@ impl DataSource for FileScanConfig { support: FilterPushdownSupport::Supported { child_descriptions, op: new_data_source, - retry, + revisit: retry, }, remaining_description, }) diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index ce8ba81a41676..3de2719421cbf 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -208,7 +208,6 @@ impl ExecutionPlan for DataSourceExec { fn try_pushdown_filters( &self, - _node: Arc, fd: FilterDescription, config: &ConfigOptions, ) -> Result>> { @@ -221,7 +220,7 @@ impl ExecutionPlan for DataSourceExec { FilterPushdownSupport::Supported { child_descriptions, op, - retry, + revisit: retry, } => { let new_exec = Arc::new(DataSourceExec::new(op)); @@ -232,7 +231,7 @@ impl ExecutionPlan for DataSourceExec { support: FilterPushdownSupport::Supported { child_descriptions, op: new_exec, - retry, + revisit: retry, }, remaining_description, }) diff --git a/datafusion/physical-optimizer/src/filter_pushdown.rs b/datafusion/physical-optimizer/src/filter_pushdown.rs index 63999f3579c61..fe02184f7bfb2 100644 --- a/datafusion/physical-optimizer/src/filter_pushdown.rs +++ b/datafusion/physical-optimizer/src/filter_pushdown.rs @@ -402,7 +402,6 @@ impl PushdownFilter { mut node: FilterDescriptionContext, config: &ConfigOptions, ) -> Result> { - let initial_plan = Arc::clone(&node.plan); let initial_description = FilterDescription { filters: node.data.take_description(), }; @@ -410,41 +409,38 @@ impl PushdownFilter { let FilterPushdownResult { support, remaining_description, - } = initial_plan.try_pushdown_filters( - Arc::clone(&initial_plan), - initial_description, - config, - )?; + } = node + .plan + .try_pushdown_filters(initial_description, config)?; match support { FilterPushdownSupport::Supported { mut child_descriptions, op, - retry, + revisit, } => { - if retry { + if revisit { // This check handles cases where the current operator is entirely removed // from the plan and replaced with its child. In such cases, to not skip // over the new node, we need to explicitly re-apply this pushdown logic // to the new node. // - // TODO: If TreeNodeRecursion supports a retry mechanism in the future, + // TODO: If TreeNodeRecursion supports a Revisit mechanism in the future, // this manual recursion could be removed. // If the operator is removed, it should not leave any filters as remaining debug_assert!(remaining_description.filters.is_empty()); - node.plan = op; // Operators having 2 children cannot be removed + debug_assert_eq!(child_descriptions.len(), 1); + debug_assert_eq!(node.children.len(), 1); + + node.plan = op; node.data = child_descriptions.swap_remove(0); - if node.children.is_empty() { - node.children = vec![]; - Ok(Transformed::yes(node)) - } else { - node.children = node.children.swap_remove(0).children; - Self::try_pushdown(node, config) - } + node.children = node.children.swap_remove(0).children; + Self::try_pushdown(node, config) } else { if remaining_description.filters.is_empty() { + // Filter can be pushed down safely node.plan = op; for (child, descr) in node.children.iter_mut().zip(child_descriptions) @@ -452,6 +448,7 @@ impl PushdownFilter { child.data = descr; } } else { + // Filter cannot be pushed down node = insert_filter_exec( node, child_descriptions, @@ -468,7 +465,7 @@ impl PushdownFilter { let children_len = node.children.len(); node = insert_filter_exec( node, - vec![FilterDescription::default(); children_len], + vec![FilterDescription::empty(); children_len], remaining_description, )?; Ok(Transformed::yes(node)) @@ -487,6 +484,7 @@ fn insert_filter_exec( // Filter has one child if !child_descriptions.is_empty() { + debug_assert_eq!(child_descriptions.len(), 1); new_child_node.data = child_descriptions.swap_remove(0); } let new_plan = Arc::new(FilterExec::try_new( @@ -494,7 +492,7 @@ fn insert_filter_exec( Arc::clone(&new_child_node.plan), )?); let new_children = vec![new_child_node]; - let new_data = FilterDescription::default(); + let new_data = FilterDescription::empty(); Ok(FilterDescriptionContext::new( new_plan, diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index a73a674081f6e..faab5fdc5eb6c 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -36,7 +36,7 @@ use datafusion_execution::TaskContext; use crate::coalesce::{BatchCoalescer, CoalescerState}; use crate::execution_plan::CardinalityEffect; use crate::filter_pushdown::{ - FilterDescription, FilterPushdownResult, FilterPushdownSupport, + filter_pushdown_transparent, FilterDescription, FilterPushdownResult, }; use datafusion_common::config::ConfigOptions; use futures::ready; @@ -219,21 +219,13 @@ impl ExecutionPlan for CoalesceBatchesExec { fn try_pushdown_filters( &self, - node: Arc, fd: FilterDescription, _config: &ConfigOptions, ) -> Result>> { - let child_descriptions = vec![fd]; - let remaining_description = FilterDescription::empty(); - - Ok(FilterPushdownResult { - support: FilterPushdownSupport::Supported { - child_descriptions, - op: Arc::clone(&node), - retry: false, - }, - remaining_description, - }) + Ok(filter_pushdown_transparent::>( + Arc::new(self.clone()), + fd, + )) } } diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index 1dd9cb1e2ea8e..e508e8ad0fe80 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -484,7 +484,6 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// [`PushdownFilter`]: datafusion_physical_optimizer::filter_pushdown::PushdownFilter fn try_pushdown_filters( &self, - _node: Arc, fd: FilterDescription, _config: &ConfigOptions, ) -> Result>> { diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 7f5a93fc0a567..95fa67025e90d 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -440,48 +440,48 @@ impl ExecutionPlan for FilterExec { fn try_pushdown_filters( &self, - node: Arc, mut fd: FilterDescription, _config: &ConfigOptions, ) -> Result>> { + // Extend the filter descriptions fd.filters.push(Arc::clone(&self.predicate)); + + // Extract the information let child_descriptions = vec![fd]; let remaining_description = FilterDescription { filters: vec![] }; - let filter_child = Arc::clone(node.children()[0]); - - if self.projection.is_some() { - let filter_child_schema = filter_child.schema(); - let proj_exprs = self - .projection - .as_ref() - .unwrap() + let filter_input = Arc::clone(self.input()); + + if let Some(projection_indices) = self.projection.as_ref() { + // Push the filters down, but leave a ProjectionExec behind, instead of the FilterExec + let filter_child_schema = filter_input.schema(); + let proj_exprs = projection_indices .iter() .map(|p| { - let col_name = filter_child_schema.field(*p).clone(); + let field = filter_child_schema.field(*p).clone(); ( - Arc::new(Column::new(col_name.name(), *p)) - as Arc, - col_name.name().to_string(), + Arc::new(Column::new(field.name(), *p)) as Arc, + field.name().to_string(), ) }) .collect::>(); let projection_exec = - Arc::new(ProjectionExec::try_new(proj_exprs, filter_child)?) as _; + Arc::new(ProjectionExec::try_new(proj_exprs, filter_input)?) as _; Ok(FilterPushdownResult { support: FilterPushdownSupport::Supported { child_descriptions, op: projection_exec, - retry: false, + revisit: false, }, remaining_description, }) } else { + // Pull out the FilterExec, and inform the rule as it should be re-run Ok(FilterPushdownResult { support: FilterPushdownSupport::Supported { child_descriptions, - op: filter_child, - retry: true, + op: filter_input, + revisit: true, }, remaining_description, }) diff --git a/datafusion/physical-plan/src/filter_pushdown.rs b/datafusion/physical-plan/src/filter_pushdown.rs index ba11aca704c51..38f5aef5923e1 100644 --- a/datafusion/physical-plan/src/filter_pushdown.rs +++ b/datafusion/physical-plan/src/filter_pushdown.rs @@ -17,6 +17,7 @@ use std::sync::Arc; +use crate::ExecutionPlan; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; #[derive(Clone, Debug)] @@ -52,8 +53,8 @@ pub enum FilterPushdownSupport { op: T, // Whether the node is removed from the plan and the rule should be re-run manually // on the new node. - // TODO: If TreeNodeRecursion supports Retry mechanism, this flag can be removed - retry: bool, + // TODO: If TreeNodeRecursion supports Revisit mechanism, this flag can be removed + revisit: bool, }, NotSupported, } @@ -75,3 +76,20 @@ pub fn filter_pushdown_not_supported( remaining_description, } } + +pub fn filter_pushdown_transparent( + plan: Arc, + fd: FilterDescription, +) -> FilterPushdownResult> { + let child_descriptions = vec![fd]; + let remaining_description = FilterDescription::empty(); + + FilterPushdownResult { + support: FilterPushdownSupport::Supported { + child_descriptions, + op: plan, + revisit: false, + }, + remaining_description, + } +} diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index bff13d580f8ec..c480fc2abaa1a 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -54,7 +54,7 @@ use datafusion_physical_expr::{EquivalenceProperties, PhysicalExpr}; use datafusion_physical_expr_common::sort_expr::LexOrdering; use crate::filter_pushdown::{ - FilterDescription, FilterPushdownResult, FilterPushdownSupport, + filter_pushdown_transparent, FilterDescription, FilterPushdownResult, }; use futures::stream::Stream; use futures::{FutureExt, StreamExt, TryStreamExt}; @@ -737,21 +737,13 @@ impl ExecutionPlan for RepartitionExec { fn try_pushdown_filters( &self, - node: Arc, fd: FilterDescription, _config: &ConfigOptions, ) -> Result>> { - let child_descriptions = vec![fd]; - let remaining_description = FilterDescription::empty(); - - Ok(FilterPushdownResult { - support: FilterPushdownSupport::Supported { - child_descriptions, - op: Arc::clone(&node), - retry: false, - }, - remaining_description, - }) + Ok(filter_pushdown_transparent::>( + Arc::new(self.clone()), + fd, + )) } } From 367377f73269dc60794934f96ab39d4421a07959 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Thu, 17 Apr 2025 15:23:29 +0300 Subject: [PATCH 36/40] apply stop logic --- .../physical_optimizer/filter_pushdown.rs | 71 +++++++++++-------- .../physical-optimizer/src/filter_pushdown.rs | 52 +++++++++++--- 2 files changed, 83 insertions(+), 40 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs index 5fbba010cc36a..4cef030e35cb6 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs @@ -154,16 +154,21 @@ impl FileSource for TestSource { fn try_pushdown_filters( &self, - fd: FilterDescription, + mut fd: FilterDescription, config: &ConfigOptions, ) -> Result>> { if self.support && config.execution.parquet.pushdown_filters { + if let Some(internal) = self.predicate.as_ref() { + fd.filters.push(Arc::clone(internal)); + } + let all_filters = fd.take_description(); + Ok(FilterPushdownResult { support: FilterPushdownSupport::Supported { child_descriptions: vec![], op: Arc::new(TestSource { support: true, - predicate: Some(conjunction(fd.filters)), + predicate: Some(conjunction(all_filters)), statistics: self.statistics.clone(), // should be updated in reality }), revisit: false, @@ -201,10 +206,10 @@ fn test_pushdown_into_scan() { OptimizationTest: input: - FilterExec: a@0 = foo - - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, pushdown_supported=true output: Ok: - - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, predicate=a@0 = foo + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=a@0 = foo " ); } @@ -212,7 +217,7 @@ fn test_pushdown_into_scan() { /// Show that we can use config options to determine how to do pushdown. #[test] fn test_pushdown_into_scan_with_config_options() { - let scan = test_scan(false); + let scan = test_scan(true); let predicate = col_lit_predicate("a", "foo", schema()); let plan = Arc::new(FilterExec::try_new(predicate, scan).unwrap()) as _; @@ -227,11 +232,11 @@ fn test_pushdown_into_scan_with_config_options() { OptimizationTest: input: - FilterExec: a@0 = foo - - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, pushdown_supported=true output: Ok: - FilterExec: a@0 = foo - - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, pushdown_supported=true " ); @@ -246,11 +251,10 @@ fn test_pushdown_into_scan_with_config_options() { OptimizationTest: input: - FilterExec: a@0 = foo - - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, pushdown_supported=true output: Ok: - - FilterExec: a@0 = foo - - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=a@0 = foo " ); } @@ -271,10 +275,10 @@ fn test_filter_collapse() { input: - FilterExec: b@1 = bar - FilterExec: a@0 = foo - - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, pushdown_supported=true output: Ok: - - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, predicate=b@1 = bar AND a@0 = foo + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=b@1 = bar AND a@0 = foo " ); } @@ -297,12 +301,12 @@ fn test_filter_with_projection() { @r" OptimizationTest: input: - - FilterExec: a@1 = foo, projection=[b@1, a@0] - - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test + - FilterExec: a@0 = foo, projection=[b@1, a@0] + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, pushdown_supported=true output: Ok: - ProjectionExec: expr=[b@1 as b, a@0 as a] - - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, predicate=a@1 = foo + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=a@0 = foo ", ); @@ -321,11 +325,11 @@ fn test_filter_with_projection() { OptimizationTest: input: - FilterExec: a@0 = foo, projection=[b@1] - - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, pushdown_supported=true output: Ok: - ProjectionExec: expr=[b@1 as b] - - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, predicate=a@0 = foo + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=a@0 = foo " ); } @@ -340,7 +344,7 @@ fn test_push_down_through_transparent_nodes() { let repartition = Arc::new( RepartitionExec::try_new(filter, Partitioning::RoundRobinBatch(1)).unwrap(), ); - let predicate = col_lit_predicate("a", "bar", schema()); + let predicate = col_lit_predicate("b", "bar", schema()); let plan = Arc::new(FilterExec::try_new(predicate, repartition).unwrap()); // expect the predicate to be pushed down into the DataSource @@ -349,16 +353,16 @@ fn test_push_down_through_transparent_nodes() { @r" OptimizationTest: input: - - FilterExec: a@0 = bar + - FilterExec: b@1 = bar - RepartitionExec: partitioning=RoundRobinBatch(1), input_partitions=0 - FilterExec: a@0 = foo - CoalesceBatchesExec: target_batch_size=1 - - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, pushdown_supported=true output: Ok: - RepartitionExec: partitioning=RoundRobinBatch(1), input_partitions=0 - CoalesceBatchesExec: target_batch_size=1 - - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, predicate=a@0 = bar AND a@0 = foo + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=b@1 = bar AND a@0 = foo " ); } @@ -371,9 +375,10 @@ fn test_no_pushdown_through_aggregates() { // 2. The inner filter **is** pushed down into the DataSource. let scan = test_scan(true); + let coalesce = Arc::new(CoalesceBatchesExec::new(scan, 10)); + let filter = Arc::new( - FilterExec::try_new(col_lit_predicate("a", "foo", schema()), scan.clone()) - .unwrap(), + FilterExec::try_new(col_lit_predicate("a", "foo", schema()), coalesce).unwrap(), ); let aggregate_expr = @@ -403,7 +408,7 @@ fn test_no_pushdown_through_aggregates() { let coalesce = Arc::new(CoalesceBatchesExec::new(aggregate, 100)); - let predicate = col_lit_predicate("a", "foo", schema()); + let predicate = col_lit_predicate("b", "bar", schema()); let plan = Arc::new(FilterExec::try_new(predicate, coalesce).unwrap()); // expect the predicate to be pushed down into the DataSource @@ -412,15 +417,19 @@ fn test_no_pushdown_through_aggregates() { @r" OptimizationTest: input: - - FilterExec: a@0 = foo - - AggregateExec: mode=Final, gby=[a@0 as a, b@1 as b], aggr=[cnt], ordering_mode=PartiallySorted([0]) - - FilterExec: a@0 = foo - - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test + - FilterExec: b@1 = bar + - CoalesceBatchesExec: target_batch_size=100 + - AggregateExec: mode=Final, gby=[a@0 as a, b@1 as b], aggr=[cnt], ordering_mode=PartiallySorted([0]) + - FilterExec: a@0 = foo + - CoalesceBatchesExec: target_batch_size=10 + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, pushdown_supported=true output: Ok: - - FilterExec: a@0 = foo - - AggregateExec: mode=Final, gby=[a@0 as a, b@1 as b], aggr=[cnt] - - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, predicate=a@0 = foo + - FilterExec: b@1 = bar + - CoalesceBatchesExec: target_batch_size=100 + - AggregateExec: mode=Final, gby=[a@0 as a, b@1 as b], aggr=[cnt] + - CoalesceBatchesExec: target_batch_size=10 + - DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=a@0 = foo " ); } diff --git a/datafusion/physical-optimizer/src/filter_pushdown.rs b/datafusion/physical-optimizer/src/filter_pushdown.rs index fe02184f7bfb2..560fbdfc00320 100644 --- a/datafusion/physical-optimizer/src/filter_pushdown.rs +++ b/datafusion/physical-optimizer/src/filter_pushdown.rs @@ -18,7 +18,8 @@ use std::sync::Arc; use crate::PhysicalOptimizerRule; -use datafusion_common::tree_node::{Transformed, TreeNode}; + +use datafusion_common::tree_node::{Transformed, TreeNode, TreeNodeRecursion}; use datafusion_common::{config::ConfigOptions, Result}; use datafusion_physical_expr::conjunction; use datafusion_physical_plan::filter::FilterExec; @@ -380,7 +381,27 @@ impl PhysicalOptimizerRule for PushdownFilter { let context = FilterDescriptionContext::new_default(plan); context - .transform_down(|node| Self::try_pushdown(node, config)) + .transform_up(|node| { + if node.plan.as_any().downcast_ref::().is_some() { + let initial_plan = Arc::clone(&node.plan); + let mut accept_updated = false; + let updated_node = node.transform_down(|filter_node| { + Self::try_pushdown(filter_node, config, &mut accept_updated) + }); + + if accept_updated { + updated_node + } else { + Ok(Transformed::no(FilterDescriptionContext::new_default( + initial_plan, + ))) + } + } + // Other filter introducing operators extends here + else { + Ok(Transformed::no(node)) + } + }) .map(|updated| updated.data.plan) } @@ -401,6 +422,7 @@ impl PushdownFilter { fn try_pushdown( mut node: FilterDescriptionContext, config: &ConfigOptions, + accept_updated: &mut bool, ) -> Result> { let initial_description = FilterDescription { filters: node.data.take_description(), @@ -437,15 +459,19 @@ impl PushdownFilter { node.plan = op; node.data = child_descriptions.swap_remove(0); node.children = node.children.swap_remove(0).children; - Self::try_pushdown(node, config) + Self::try_pushdown(node, config, accept_updated) } else { if remaining_description.filters.is_empty() { // Filter can be pushed down safely node.plan = op; - for (child, descr) in - node.children.iter_mut().zip(child_descriptions) - { - child.data = descr; + if node.children.is_empty() { + *accept_updated = true; + } else { + for (child, descr) in + node.children.iter_mut().zip(child_descriptions) + { + child.data = descr; + } } } else { // Filter cannot be pushed down @@ -460,7 +486,11 @@ impl PushdownFilter { } FilterPushdownSupport::NotSupported => { if remaining_description.filters.is_empty() { - Ok(Transformed::no(node)) + Ok(Transformed { + data: node, + transformed: false, + tnr: TreeNodeRecursion::Stop, + }) } else { let children_len = node.children.len(); node = insert_filter_exec( @@ -468,7 +498,11 @@ impl PushdownFilter { vec![FilterDescription::empty(); children_len], remaining_description, )?; - Ok(Transformed::yes(node)) + Ok(Transformed { + data: node, + transformed: true, + tnr: TreeNodeRecursion::Stop, + }) } } } From 616165d6ad8433c5723752e61bc8ac7dcf431260 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Thu, 17 Apr 2025 15:56:13 +0300 Subject: [PATCH 37/40] update slt's --- .../physical-optimizer/src/filter_pushdown.rs | 3 +- .../sqllogictest/test_files/aggregate.slt | 4 +- datafusion/sqllogictest/test_files/array.slt | 24 +- datafusion/sqllogictest/test_files/cte.slt | 20 +- .../sqllogictest/test_files/explain.slt | 4 +- .../sqllogictest/test_files/explain_tree.slt | 444 +++++++++--------- .../test_files/filter_without_sort_exec.slt | 24 +- .../sqllogictest/test_files/join.slt.part | 6 +- .../join_disable_repartition_joins.slt | 4 +- datafusion/sqllogictest/test_files/joins.slt | 24 +- .../sqllogictest/test_files/parquet.slt | 16 +- .../test_files/parquet_filter_pushdown.slt | 4 +- .../sqllogictest/test_files/predicates.slt | 12 +- .../sqllogictest/test_files/repartition.slt | 4 +- datafusion/sqllogictest/test_files/select.slt | 24 +- .../sqllogictest/test_files/simplify_expr.slt | 1 - .../sqllogictest/test_files/subquery.slt | 11 +- datafusion/sqllogictest/test_files/union.slt | 12 +- datafusion/sqllogictest/test_files/window.slt | 16 +- 19 files changed, 327 insertions(+), 330 deletions(-) diff --git a/datafusion/physical-optimizer/src/filter_pushdown.rs b/datafusion/physical-optimizer/src/filter_pushdown.rs index 560fbdfc00320..c66f3278f6763 100644 --- a/datafusion/physical-optimizer/src/filter_pushdown.rs +++ b/datafusion/physical-optimizer/src/filter_pushdown.rs @@ -492,10 +492,9 @@ impl PushdownFilter { tnr: TreeNodeRecursion::Stop, }) } else { - let children_len = node.children.len(); node = insert_filter_exec( node, - vec![FilterDescription::empty(); children_len], + vec![FilterDescription::empty(); 1], remaining_description, )?; Ok(Transformed { diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index f906037f8d231..35bbc758141b6 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -5213,8 +5213,8 @@ physical_plan 08)--------------CoalescePartitionsExec 09)----------------AggregateExec: mode=Partial, gby=[c2@0 as c2, c3@1 as c3], aggr=[] 10)------------------CoalesceBatchesExec: target_batch_size=8192 -11)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -12)----------------------FilterExec: c3@1 >= 10 AND c3@1 <= 20 +11)--------------------FilterExec: c3@1 >= 10 AND c3@1 <= 20 +12)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 13)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], file_type=csv, has_header=true query I diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index 69cb9103caca1..e780d6c8b295a 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -6002,8 +6002,8 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] 05)--------ProjectionExec: expr=[] 06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------FilterExec: substr(md5(CAST(value@0 AS Utf8)), 1, 32) IN ([Literal { value: Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278") }, Literal { value: Utf8View("a") }, Literal { value: Utf8View("b") }, Literal { value: Utf8View("c") }]) +07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8)), 1, 32) IN ([Literal { value: Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278") }, Literal { value: Utf8View("a") }, Literal { value: Utf8View("b") }, Literal { value: Utf8View("c") }]) +08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] query I @@ -6031,8 +6031,8 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] 05)--------ProjectionExec: expr=[] 06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------FilterExec: substr(md5(CAST(value@0 AS Utf8)), 1, 32) IN ([Literal { value: Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278") }, Literal { value: Utf8View("a") }, Literal { value: Utf8View("b") }, Literal { value: Utf8View("c") }]) +07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8)), 1, 32) IN ([Literal { value: Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278") }, Literal { value: Utf8View("a") }, Literal { value: Utf8View("b") }, Literal { value: Utf8View("c") }]) +08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] query I @@ -6060,8 +6060,8 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] 05)--------ProjectionExec: expr=[] 06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------FilterExec: substr(md5(CAST(value@0 AS Utf8)), 1, 32) IN ([Literal { value: Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278") }, Literal { value: Utf8View("a") }, Literal { value: Utf8View("b") }, Literal { value: Utf8View("c") }]) +07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8)), 1, 32) IN ([Literal { value: Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278") }, Literal { value: Utf8View("a") }, Literal { value: Utf8View("b") }, Literal { value: Utf8View("c") }]) +08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] # FIXME: due to rewrite below not working, this is _extremely_ slow to evaluate @@ -6091,8 +6091,8 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] 05)--------ProjectionExec: expr=[] 06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------FilterExec: array_has([7f4b18de3cfeb9b4ac78c381ee2ad278, a, b, c], substr(md5(CAST(value@0 AS Utf8)), 1, 32)) +07)------------FilterExec: array_has([7f4b18de3cfeb9b4ac78c381ee2ad278, a, b, c], substr(md5(CAST(value@0 AS Utf8)), 1, 32)) +08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] query I @@ -6120,8 +6120,8 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] 05)--------ProjectionExec: expr=[] 06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------FilterExec: substr(md5(CAST(value@0 AS Utf8)), 1, 32) IN ([Literal { value: Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278") }, Literal { value: Utf8View("a") }, Literal { value: Utf8View("b") }, Literal { value: Utf8View("c") }]) +07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8)), 1, 32) IN ([Literal { value: Utf8View("7f4b18de3cfeb9b4ac78c381ee2ad278") }, Literal { value: Utf8View("a") }, Literal { value: Utf8View("b") }, Literal { value: Utf8View("c") }]) +08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] query I @@ -6151,8 +6151,8 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] 05)--------ProjectionExec: expr=[] 06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------FilterExec: substr(md5(CAST(value@0 AS Utf8)), 1, 32) IS NOT NULL OR NULL +07)------------FilterExec: substr(md5(CAST(value@0 AS Utf8)), 1, 32) IS NOT NULL OR NULL +08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192] # any operator diff --git a/datafusion/sqllogictest/test_files/cte.slt b/datafusion/sqllogictest/test_files/cte.slt index 20930ec6a4c58..32320a06f4fb0 100644 --- a/datafusion/sqllogictest/test_files/cte.slt +++ b/datafusion/sqllogictest/test_files/cte.slt @@ -118,8 +118,8 @@ physical_plan 04)--CoalescePartitionsExec 05)----ProjectionExec: expr=[id@0 + 1 as id] 06)------CoalesceBatchesExec: target_batch_size=8192 -07)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)----------FilterExec: id@0 < 10 +07)--------FilterExec: id@0 < 10 +08)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)------------WorkTableExec: name=nodes # setup @@ -164,8 +164,8 @@ physical_plan 04)----CoalescePartitionsExec 05)------ProjectionExec: expr=[time@0 + 1 as time, name@1 as name, account_balance@2 + 10 as account_balance] 06)--------CoalesceBatchesExec: target_batch_size=2 -07)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)------------FilterExec: time@0 < 10 +07)----------FilterExec: time@0 < 10 +08)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)--------------WorkTableExec: name=balances # recursive CTE with static term derived from table works @@ -736,8 +736,8 @@ physical_plan 05)----CrossJoinExec 06)------CoalescePartitionsExec 07)--------CoalesceBatchesExec: target_batch_size=8182 -08)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)------------FilterExec: val@0 < 2 +08)----------FilterExec: val@0 < 2 +09)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 10)--------------WorkTableExec: name=recursive_cte 11)------ProjectionExec: expr=[2 as val] 12)--------PlaceholderRowExec @@ -966,8 +966,8 @@ physical_plan 04)--CoalescePartitionsExec 05)----ProjectionExec: expr=[n@0 + 1 as numbers.n + Int64(1)] 06)------CoalesceBatchesExec: target_batch_size=8182 -07)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)----------FilterExec: n@0 < 10 +07)--------FilterExec: n@0 < 10 +08)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)------------WorkTableExec: name=numbers query TT @@ -992,8 +992,8 @@ physical_plan 04)--CoalescePartitionsExec 05)----ProjectionExec: expr=[n@0 + 1 as numbers.n + Int64(1)] 06)------CoalesceBatchesExec: target_batch_size=8182 -07)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)----------FilterExec: n@0 < 10 +07)--------FilterExec: n@0 < 10 +08)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)------------WorkTableExec: name=numbers statement count 0 diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 8d0753090c4d9..ba2596551f1d5 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -44,8 +44,8 @@ logical_plan 03)----TableScan: aggregate_test_100 projection=[c1, c2], partial_filters=[aggregate_test_100.c2 > Int8(10)] physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 -02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -03)----FilterExec: c2@1 > 10, projection=[c1@0] +02)--FilterExec: c2@1 > 10, projection=[c1@0] +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2], file_type=csv, has_header=true # explain_csv_exec_scan_config diff --git a/datafusion/sqllogictest/test_files/explain_tree.slt b/datafusion/sqllogictest/test_files/explain_tree.slt index 6973c9a7de2df..15bf615765713 100644 --- a/datafusion/sqllogictest/test_files/explain_tree.slt +++ b/datafusion/sqllogictest/test_files/explain_tree.slt @@ -172,19 +172,19 @@ physical_plan 05)│ 8192 │ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ RepartitionExec │ +08)│ FilterExec │ 09)│ -------------------- │ -10)│ partition_count(in->out): │ -11)│ 1 -> 4 │ -12)│ │ -13)│ partitioning_scheme: │ -14)│ RoundRobinBatch(4) │ -15)└─────────────┬─────────────┘ -16)┌─────────────┴─────────────┐ -17)│ FilterExec │ -18)│ -------------------- │ -19)│ predicate: │ -20)│ string_col != foo │ +10)│ predicate: │ +11)│ string_col != foo │ +12)└─────────────┬─────────────┘ +13)┌─────────────┴─────────────┐ +14)│ RepartitionExec │ +15)│ -------------------- │ +16)│ partition_count(in->out): │ +17)│ 1 -> 4 │ +18)│ │ +19)│ partitioning_scheme: │ +20)│ RoundRobinBatch(4) │ 21)└─────────────┬─────────────┘ 22)┌─────────────┴─────────────┐ 23)│ DataSourceExec │ @@ -422,23 +422,23 @@ physical_plan 05)│ 8192 │ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ RepartitionExec │ +08)│ FilterExec │ 09)│ -------------------- │ -10)│ partition_count(in->out): │ -11)│ 1 -> 4 │ -12)│ │ -13)│ partitioning_scheme: │ -14)│ RoundRobinBatch(4) │ -15)└─────────────┬─────────────┘ -16)┌─────────────┴─────────────┐ -17)│ FilterExec │ -18)│ -------------------- │ -19)│ predicate: │ -20)│ string_col != foo AND │ -21)│ string_col != bar │ -22)│ AND string_col != a │ -23)│ really long string │ -24)│ constant │ +10)│ predicate: │ +11)│ string_col != foo AND │ +12)│ string_col != bar │ +13)│ AND string_col != a │ +14)│ really long string │ +15)│ constant │ +16)└─────────────┬─────────────┘ +17)┌─────────────┴─────────────┐ +18)│ RepartitionExec │ +19)│ -------------------- │ +20)│ partition_count(in->out): │ +21)│ 1 -> 4 │ +22)│ │ +23)│ partitioning_scheme: │ +24)│ RoundRobinBatch(4) │ 25)└─────────────┬─────────────┘ 26)┌─────────────┴─────────────┐ 27)│ DataSourceExec │ @@ -460,20 +460,20 @@ physical_plan 05)│ 8192 │ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ RepartitionExec │ +08)│ FilterExec │ 09)│ -------------------- │ -10)│ partition_count(in->out): │ -11)│ 1 -> 4 │ -12)│ │ -13)│ partitioning_scheme: │ -14)│ RoundRobinBatch(4) │ -15)└─────────────┬─────────────┘ -16)┌─────────────┴─────────────┐ -17)│ FilterExec │ -18)│ -------------------- │ -19)│ predicate: │ -20)│ string_col != │ -21)│ aaaaaaaaaaaa │ +10)│ predicate: │ +11)│ string_col != │ +12)│ aaaaaaaaaaaa │ +13)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +14)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +15)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +16)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +17)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +18)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +19)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +20)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +21)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ 22)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ 23)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ 24)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ @@ -491,16 +491,16 @@ physical_plan 36)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ 37)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ 38)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ -39)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ -40)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ -41)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ -42)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ -43)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ -44)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ -45)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ -46)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ -47)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ -48)│ ... │ +39)│ ... │ +40)└─────────────┬─────────────┘ +41)┌─────────────┴─────────────┐ +42)│ RepartitionExec │ +43)│ -------------------- │ +44)│ partition_count(in->out): │ +45)│ 1 -> 4 │ +46)│ │ +47)│ partitioning_scheme: │ +48)│ RoundRobinBatch(4) │ 49)└─────────────┬─────────────┘ 50)┌─────────────┴─────────────┐ 51)│ DataSourceExec │ @@ -522,19 +522,19 @@ physical_plan 05)│ 8192 │ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ RepartitionExec │ +08)│ FilterExec │ 09)│ -------------------- │ -10)│ partition_count(in->out): │ -11)│ 1 -> 4 │ -12)│ │ -13)│ partitioning_scheme: │ -14)│ RoundRobinBatch(4) │ -15)└─────────────┬─────────────┘ -16)┌─────────────┴─────────────┐ -17)│ FilterExec │ -18)│ -------------------- │ -19)│ predicate: │ -20)│string_col != aaaaaaaaaaaaa│ +10)│ predicate: │ +11)│string_col != aaaaaaaaaaaaa│ +12)└─────────────┬─────────────┘ +13)┌─────────────┴─────────────┐ +14)│ RepartitionExec │ +15)│ -------------------- │ +16)│ partition_count(in->out): │ +17)│ 1 -> 4 │ +18)│ │ +19)│ partitioning_scheme: │ +20)│ RoundRobinBatch(4) │ 21)└─────────────┬─────────────┘ 22)┌─────────────┴─────────────┐ 23)│ DataSourceExec │ @@ -556,21 +556,21 @@ physical_plan 05)│ 8192 │ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ RepartitionExec │ +08)│ FilterExec │ 09)│ -------------------- │ -10)│ partition_count(in->out): │ -11)│ 1 -> 4 │ -12)│ │ -13)│ partitioning_scheme: │ -14)│ RoundRobinBatch(4) │ -15)└─────────────┬─────────────┘ -16)┌─────────────┴─────────────┐ -17)│ FilterExec │ -18)│ -------------------- │ -19)│ predicate: │ -20)│ string_col != │ -21)│ aaaaaaaaaaaa │ -22)│ aaa │ +10)│ predicate: │ +11)│ string_col != │ +12)│ aaaaaaaaaaaa │ +13)│ aaa │ +14)└─────────────┬─────────────┘ +15)┌─────────────┴─────────────┐ +16)│ RepartitionExec │ +17)│ -------------------- │ +18)│ partition_count(in->out): │ +19)│ 1 -> 4 │ +20)│ │ +21)│ partitioning_scheme: │ +22)│ RoundRobinBatch(4) │ 23)└─────────────┬─────────────┘ 24)┌─────────────┴─────────────┐ 25)│ DataSourceExec │ @@ -591,19 +591,19 @@ physical_plan 05)│ 8192 │ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ RepartitionExec │ +08)│ FilterExec │ 09)│ -------------------- │ -10)│ partition_count(in->out): │ -11)│ 1 -> 4 │ -12)│ │ -13)│ partitioning_scheme: │ -14)│ RoundRobinBatch(4) │ -15)└─────────────┬─────────────┘ -16)┌─────────────┴─────────────┐ -17)│ FilterExec │ -18)│ -------------------- │ -19)│ predicate: │ -20)│ string_col != foo │ +10)│ predicate: │ +11)│ string_col != foo │ +12)└─────────────┬─────────────┘ +13)┌─────────────┴─────────────┐ +14)│ RepartitionExec │ +15)│ -------------------- │ +16)│ partition_count(in->out): │ +17)│ 1 -> 4 │ +18)│ │ +19)│ partitioning_scheme: │ +20)│ RoundRobinBatch(4) │ 21)└─────────────┬─────────────┘ 22)┌─────────────┴─────────────┐ 23)│ DataSourceExec │ @@ -625,19 +625,19 @@ physical_plan 05)│ 8192 │ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ RepartitionExec │ +08)│ FilterExec │ 09)│ -------------------- │ -10)│ partition_count(in->out): │ -11)│ 1 -> 4 │ -12)│ │ -13)│ partitioning_scheme: │ -14)│ RoundRobinBatch(4) │ -15)└─────────────┬─────────────┘ -16)┌─────────────┴─────────────┐ -17)│ FilterExec │ -18)│ -------------------- │ -19)│ predicate: │ -20)│ string_col != foo │ +10)│ predicate: │ +11)│ string_col != foo │ +12)└─────────────┬─────────────┘ +13)┌─────────────┴─────────────┐ +14)│ RepartitionExec │ +15)│ -------------------- │ +16)│ partition_count(in->out): │ +17)│ 1 -> 4 │ +18)│ │ +19)│ partitioning_scheme: │ +20)│ RoundRobinBatch(4) │ 21)└─────────────┬─────────────┘ 22)┌─────────────┴─────────────┐ 23)│ DataSourceExec │ @@ -686,19 +686,19 @@ physical_plan 05)│ 8192 │ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ RepartitionExec │ +08)│ FilterExec │ 09)│ -------------------- │ -10)│ partition_count(in->out): │ -11)│ 1 -> 4 │ -12)│ │ -13)│ partitioning_scheme: │ -14)│ RoundRobinBatch(4) │ -15)└─────────────┬─────────────┘ -16)┌─────────────┴─────────────┐ -17)│ FilterExec │ -18)│ -------------------- │ -19)│ predicate: │ -20)│ string_col != foo │ +10)│ predicate: │ +11)│ string_col != foo │ +12)└─────────────┬─────────────┘ +13)┌─────────────┴─────────────┐ +14)│ RepartitionExec │ +15)│ -------------------- │ +16)│ partition_count(in->out): │ +17)│ 1 -> 4 │ +18)│ │ +19)│ partitioning_scheme: │ +20)│ RoundRobinBatch(4) │ 21)└─────────────┬─────────────┘ 22)┌─────────────┴─────────────┐ 23)│ DataSourceExec │ @@ -719,19 +719,19 @@ physical_plan 05)│ 8192 │ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ RepartitionExec │ +08)│ FilterExec │ 09)│ -------------------- │ -10)│ partition_count(in->out): │ -11)│ 1 -> 4 │ -12)│ │ -13)│ partitioning_scheme: │ -14)│ RoundRobinBatch(4) │ -15)└─────────────┬─────────────┘ -16)┌─────────────┴─────────────┐ -17)│ FilterExec │ -18)│ -------------------- │ -19)│ predicate: │ -20)│ string_col != foo │ +10)│ predicate: │ +11)│ string_col != foo │ +12)└─────────────┬─────────────┘ +13)┌─────────────┴─────────────┐ +14)│ RepartitionExec │ +15)│ -------------------- │ +16)│ partition_count(in->out): │ +17)│ 1 -> 4 │ +18)│ │ +19)│ partitioning_scheme: │ +20)│ RoundRobinBatch(4) │ 21)└─────────────┬─────────────┘ 22)┌─────────────┴─────────────┐ 23)│ DataSourceExec │ @@ -1599,18 +1599,18 @@ physical_plan 11)│ 8192 │ 12)└─────────────┬─────────────┘ 13)┌─────────────┴─────────────┐ -14)│ RepartitionExec │ +14)│ FilterExec │ 15)│ -------------------- │ -16)│ partition_count(in->out): │ -17)│ 1 -> 4 │ -18)│ │ -19)│ partitioning_scheme: │ -20)│ RoundRobinBatch(4) │ -21)└─────────────┬─────────────┘ -22)┌─────────────┴─────────────┐ -23)│ FilterExec │ -24)│ -------------------- │ -25)│ predicate: ticker = A │ +16)│ predicate: ticker = A │ +17)└─────────────┬─────────────┘ +18)┌─────────────┴─────────────┐ +19)│ RepartitionExec │ +20)│ -------------------- │ +21)│ partition_count(in->out): │ +22)│ 1 -> 4 │ +23)│ │ +24)│ partitioning_scheme: │ +25)│ RoundRobinBatch(4) │ 26)└─────────────┬─────────────┘ 27)┌─────────────┴─────────────┐ 28)│ StreamingTableExec │ @@ -1639,20 +1639,20 @@ physical_plan 10)│ 8192 │ 11)└─────────────┬─────────────┘ 12)┌─────────────┴─────────────┐ -13)│ RepartitionExec │ +13)│ FilterExec │ 14)│ -------------------- │ -15)│ partition_count(in->out): │ -16)│ 1 -> 4 │ -17)│ │ -18)│ partitioning_scheme: │ -19)│ RoundRobinBatch(4) │ -20)└─────────────┬─────────────┘ -21)┌─────────────┴─────────────┐ -22)│ FilterExec │ -23)│ -------------------- │ -24)│ predicate: │ -25)│ ticker = A AND CAST(time │ -26)│ AS Date32) = date │ +15)│ predicate: │ +16)│ ticker = A AND CAST(time │ +17)│ AS Date32) = date │ +18)└─────────────┬─────────────┘ +19)┌─────────────┴─────────────┐ +20)│ RepartitionExec │ +21)│ -------------------- │ +22)│ partition_count(in->out): │ +23)│ 1 -> 4 │ +24)│ │ +25)│ partitioning_scheme: │ +26)│ RoundRobinBatch(4) │ 27)└─────────────┬─────────────┘ 28)┌─────────────┴─────────────┐ 29)│ StreamingTableExec │ @@ -1680,20 +1680,20 @@ physical_plan 10)│ 8192 │ 11)└─────────────┬─────────────┘ 12)┌─────────────┴─────────────┐ -13)│ RepartitionExec │ +13)│ FilterExec │ 14)│ -------------------- │ -15)│ partition_count(in->out): │ -16)│ 1 -> 4 │ -17)│ │ -18)│ partitioning_scheme: │ -19)│ RoundRobinBatch(4) │ -20)└─────────────┬─────────────┘ -21)┌─────────────┴─────────────┐ -22)│ FilterExec │ -23)│ -------------------- │ -24)│ predicate: │ -25)│ ticker = A AND CAST(time │ -26)│ AS Date32) = date │ +15)│ predicate: │ +16)│ ticker = A AND CAST(time │ +17)│ AS Date32) = date │ +18)└─────────────┬─────────────┘ +19)┌─────────────┴─────────────┐ +20)│ RepartitionExec │ +21)│ -------------------- │ +22)│ partition_count(in->out): │ +23)│ 1 -> 4 │ +24)│ │ +25)│ partitioning_scheme: │ +26)│ RoundRobinBatch(4) │ 27)└─────────────┬─────────────┘ 28)┌─────────────┴─────────────┐ 29)│ StreamingTableExec │ @@ -1719,20 +1719,20 @@ physical_plan 08)│ 8192 │ 09)└─────────────┬─────────────┘ 10)┌─────────────┴─────────────┐ -11)│ RepartitionExec │ +11)│ FilterExec │ 12)│ -------------------- │ -13)│ partition_count(in->out): │ -14)│ 1 -> 4 │ -15)│ │ -16)│ partitioning_scheme: │ -17)│ RoundRobinBatch(4) │ -18)└─────────────┬─────────────┘ -19)┌─────────────┴─────────────┐ -20)│ FilterExec │ -21)│ -------------------- │ -22)│ predicate: │ -23)│ ticker = A AND CAST(time │ -24)│ AS Date32) = date │ +13)│ predicate: │ +14)│ ticker = A AND CAST(time │ +15)│ AS Date32) = date │ +16)└─────────────┬─────────────┘ +17)┌─────────────┴─────────────┐ +18)│ RepartitionExec │ +19)│ -------------------- │ +20)│ partition_count(in->out): │ +21)│ 1 -> 4 │ +22)│ │ +23)│ partitioning_scheme: │ +24)│ RoundRobinBatch(4) │ 25)└─────────────┬─────────────┘ 26)┌─────────────┴─────────────┐ 27)│ StreamingTableExec │ @@ -1762,20 +1762,20 @@ physical_plan 11)│ 8192 │ 12)└─────────────┬─────────────┘ 13)┌─────────────┴─────────────┐ -14)│ RepartitionExec │ +14)│ FilterExec │ 15)│ -------------------- │ -16)│ partition_count(in->out): │ -17)│ 1 -> 4 │ -18)│ │ -19)│ partitioning_scheme: │ -20)│ RoundRobinBatch(4) │ -21)└─────────────┬─────────────┘ -22)┌─────────────┴─────────────┐ -23)│ FilterExec │ -24)│ -------------------- │ -25)│ predicate: │ -26)│ ticker = A AND CAST(time │ -27)│ AS Date32) = date │ +16)│ predicate: │ +17)│ ticker = A AND CAST(time │ +18)│ AS Date32) = date │ +19)└─────────────┬─────────────┘ +20)┌─────────────┴─────────────┐ +21)│ RepartitionExec │ +22)│ -------------------- │ +23)│ partition_count(in->out): │ +24)│ 1 -> 4 │ +25)│ │ +26)│ partitioning_scheme: │ +27)│ RoundRobinBatch(4) │ 28)└─────────────┬─────────────┘ 29)┌─────────────┴─────────────┐ 30)│ StreamingTableExec │ @@ -1807,19 +1807,19 @@ physical_plan 11)│ 8192 │ 12)└─────────────┬─────────────┘ 13)┌─────────────┴─────────────┐ -14)│ RepartitionExec │ +14)│ FilterExec │ 15)│ -------------------- │ -16)│ partition_count(in->out): │ -17)│ 1 -> 4 │ -18)│ │ -19)│ partitioning_scheme: │ -20)│ RoundRobinBatch(4) │ -21)└─────────────┬─────────────┘ -22)┌─────────────┴─────────────┐ -23)│ FilterExec │ -24)│ -------------------- │ -25)│ predicate: │ -26)│ date = 2006-01-02 │ +16)│ predicate: │ +17)│ date = 2006-01-02 │ +18)└─────────────┬─────────────┘ +19)┌─────────────┴─────────────┐ +20)│ RepartitionExec │ +21)│ -------------------- │ +22)│ partition_count(in->out): │ +23)│ 1 -> 4 │ +24)│ │ +25)│ partitioning_scheme: │ +26)│ RoundRobinBatch(4) │ 27)└─────────────┬─────────────┘ 28)┌─────────────┴─────────────┐ 29)│ StreamingTableExec │ @@ -1862,18 +1862,18 @@ physical_plan 18)-----------------------------│ 8192 │ 19)-----------------------------└─────────────┬─────────────┘ 20)-----------------------------┌─────────────┴─────────────┐ -21)-----------------------------│ RepartitionExec │ +21)-----------------------------│ FilterExec │ 22)-----------------------------│ -------------------- │ -23)-----------------------------│ partition_count(in->out): │ -24)-----------------------------│ 1 -> 4 │ -25)-----------------------------│ │ -26)-----------------------------│ partitioning_scheme: │ -27)-----------------------------│ RoundRobinBatch(4) │ -28)-----------------------------└─────────────┬─────────────┘ -29)-----------------------------┌─────────────┴─────────────┐ -30)-----------------------------│ FilterExec │ -31)-----------------------------│ -------------------- │ -32)-----------------------------│ predicate: id < 10 │ +23)-----------------------------│ predicate: id < 10 │ +24)-----------------------------└─────────────┬─────────────┘ +25)-----------------------------┌─────────────┴─────────────┐ +26)-----------------------------│ RepartitionExec │ +27)-----------------------------│ -------------------- │ +28)-----------------------------│ partition_count(in->out): │ +29)-----------------------------│ 1 -> 4 │ +30)-----------------------------│ │ +31)-----------------------------│ partitioning_scheme: │ +32)-----------------------------│ RoundRobinBatch(4) │ 33)-----------------------------└─────────────┬─────────────┘ 34)-----------------------------┌─────────────┴─────────────┐ 35)-----------------------------│ WorkTableExec │ @@ -2055,18 +2055,18 @@ physical_plan 12)│ 8192 │ 13)└─────────────┬─────────────┘ 14)┌─────────────┴─────────────┐ -15)│ RepartitionExec │ +15)│ FilterExec │ 16)│ -------------------- │ -17)│ partition_count(in->out): │ -18)│ 1 -> 4 │ -19)│ │ -20)│ partitioning_scheme: │ -21)│ RoundRobinBatch(4) │ -22)└─────────────┬─────────────┘ -23)┌─────────────┴─────────────┐ -24)│ FilterExec │ -25)│ -------------------- │ -26)│ predicate: c3 > 0 │ +17)│ predicate: c3 > 0 │ +18)└─────────────┬─────────────┘ +19)┌─────────────┴─────────────┐ +20)│ RepartitionExec │ +21)│ -------------------- │ +22)│ partition_count(in->out): │ +23)│ 1 -> 4 │ +24)│ │ +25)│ partitioning_scheme: │ +26)│ RoundRobinBatch(4) │ 27)└─────────────┬─────────────┘ 28)┌─────────────┴─────────────┐ 29)│ StreamingTableExec │ diff --git a/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt b/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt index c892ae17d26c6..d96044fda8c05 100644 --- a/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt +++ b/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt @@ -39,8 +39,8 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [date@0 ASC NULLS LAST, time@2 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 -03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)------FilterExec: ticker@1 = A +03)----FilterExec: ticker@1 = A +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 05)--------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] # constant ticker, CAST(time AS DATE) = time, order by time @@ -56,8 +56,8 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [time@2 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 -03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)------FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 +03)----FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 05)--------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] # same thing but order by date @@ -73,8 +73,8 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [date@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 -03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)------FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 +03)----FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 05)--------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] # same thing but order by ticker @@ -90,8 +90,8 @@ logical_plan physical_plan 01)CoalescePartitionsExec 02)--CoalesceBatchesExec: target_batch_size=8192 -03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)------FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 +03)----FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 05)--------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] # same thing but order by time, date @@ -107,8 +107,8 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [time@2 ASC NULLS LAST, date@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 -03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)------FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 +03)----FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 05)--------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] # CAST(time AS DATE) <> date (should require a sort) @@ -148,6 +148,6 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 -03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)------FilterExec: date@0 = 2006-01-02 +03)----FilterExec: date@0 = 2006-01-02 +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 05)--------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] diff --git a/datafusion/sqllogictest/test_files/join.slt.part b/datafusion/sqllogictest/test_files/join.slt.part index f4228d1b011f5..972dd2265343d 100644 --- a/datafusion/sqllogictest/test_files/join.slt.part +++ b/datafusion/sqllogictest/test_files/join.slt.part @@ -984,9 +984,9 @@ logical_plan 08)--------TableScan: department projection=[emp_id, dept_name] physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 -02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -03)----CoalesceBatchesExec: target_batch_size=8192 -04)------FilterExec: dept_name@2 != Engineering AND name@1 = Alice OR name@1 != Alice AND name@1 = Carol +02)--FilterExec: dept_name@2 != Engineering AND name@1 = Alice OR name@1 != Alice AND name@1 = Carol +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------CoalesceBatchesExec: target_batch_size=8192 05)--------HashJoinExec: mode=CollectLeft, join_type=Left, on=[(emp_id@0, emp_id@0)], projection=[emp_id@0, name@1, dept_name@3] 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------FilterExec: name@1 = Alice OR name@1 != Alice AND name@1 = Carol diff --git a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt index 898d35937d9e4..a1efc1317b4aa 100644 --- a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt +++ b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt @@ -100,8 +100,8 @@ physical_plan 04)------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d@1, d@3), (c@0, c@2)], projection=[a@0, b@1] 05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], file_type=csv, has_header=true 06)--------CoalesceBatchesExec: target_batch_size=8192 -07)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)------------FilterExec: d@3 = 3 +07)----------FilterExec: d@3 = 3 +08)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], file_type=csv, has_header=true # preserve_right_semi_join diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 7863c40154c6c..ddf701ba04efe 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -2053,12 +2053,12 @@ physical_plan 02)--NestedLoopJoinExec: join_type=Inner, filter=t1_id@0 > t2_id@1 03)----CoalescePartitionsExec 04)------CoalesceBatchesExec: target_batch_size=2 -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------FilterExec: t2_int@1 > 1, projection=[t2_id@0] +05)--------FilterExec: t2_int@1 > 1, projection=[t2_id@0] +06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 07)------------DataSourceExec: partitions=1, partition_sizes=[1] 08)----CoalesceBatchesExec: target_batch_size=2 -09)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)--------FilterExec: t1_id@0 > 10 +09)------FilterExec: t1_id@0 > 10 +10)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 11)----------DataSourceExec: partitions=1, partition_sizes=[1] query II @@ -2092,12 +2092,12 @@ physical_plan 01)NestedLoopJoinExec: join_type=Right, filter=t1_id@0 < t2_id@1 02)--CoalescePartitionsExec 03)----CoalesceBatchesExec: target_batch_size=2 -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------FilterExec: t1_id@0 > 22 +04)------FilterExec: t1_id@0 > 22 +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 06)----------DataSourceExec: partitions=1, partition_sizes=[1] 07)--CoalesceBatchesExec: target_batch_size=2 -08)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)------FilterExec: t2_id@0 > 11 +08)----FilterExec: t2_id@0 > 11 +09)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 10)--------DataSourceExec: partitions=1, partition_sizes=[1] query II @@ -4428,8 +4428,8 @@ physical_plan 03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(a@0, a@0)] 04)------CoalescePartitionsExec 05)--------CoalesceBatchesExec: target_batch_size=3 -06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)------------FilterExec: b@1 > 3, projection=[a@0] +06)----------FilterExec: b@1 > 3, projection=[a@0] +07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 08)--------------DataSourceExec: partitions=1, partition_sizes=[1] 09)------SortExec: expr=[c@2 DESC], preserve_partitioning=[true] 10)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -4452,8 +4452,8 @@ physical_plan 03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(a@0, a@0)] 04)------CoalescePartitionsExec 05)--------CoalesceBatchesExec: target_batch_size=3 -06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)------------FilterExec: b@1 > 3, projection=[a@0] +06)----------FilterExec: b@1 > 3, projection=[a@0] +07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 08)--------------DataSourceExec: partitions=1, partition_sizes=[1] 09)------SortExec: expr=[c@2 DESC NULLS LAST], preserve_partitioning=[true] 10)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 diff --git a/datafusion/sqllogictest/test_files/parquet.slt b/datafusion/sqllogictest/test_files/parquet.slt index 2d274125e0c1e..2970b2effb3e9 100644 --- a/datafusion/sqllogictest/test_files/parquet.slt +++ b/datafusion/sqllogictest/test_files/parquet.slt @@ -409,8 +409,8 @@ logical_plan 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%")] physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 -02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -03)----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)--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% +03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=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% @@ -457,8 +457,8 @@ logical_plan 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%")] physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 -02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -03)----FilterExec: binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% +02)--FilterExec: binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% +03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% @@ -508,8 +508,8 @@ logical_plan 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%")] physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 -02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -03)----FilterExec: binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% +02)--FilterExec: binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% +03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% @@ -623,8 +623,8 @@ logical_plan 02)--TableScan: foo projection=[column1], partial_filters=[foo.column1 LIKE Utf8View("f%")] physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 -02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -03)----FilterExec: column1@0 LIKE f% +02)--FilterExec: column1@0 LIKE f% +03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/foo.parquet]]}, projection=[column1], file_type=parquet, predicate=column1@0 LIKE f%, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= g AND f <= column1_max@1, required_guarantees=[] statement ok diff --git a/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt b/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt index 709376805a216..758113b708355 100644 --- a/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt +++ b/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt @@ -111,8 +111,8 @@ physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----CoalesceBatchesExec: target_batch_size=8192 -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 -05)--------FilterExec: b@1 > 2, projection=[a@0] +04)------FilterExec: b@1 > 2, projection=[a@0] +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 06)----------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, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] # also test querying on columns that are not in all the files diff --git a/datafusion/sqllogictest/test_files/predicates.slt b/datafusion/sqllogictest/test_files/predicates.slt index 070c99f67a733..b263e39f3b11b 100644 --- a/datafusion/sqllogictest/test_files/predicates.slt +++ b/datafusion/sqllogictest/test_files/predicates.slt @@ -673,14 +673,14 @@ physical_plan 03)----CoalesceBatchesExec: target_batch_size=8192 04)------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -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 +06)----------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 +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 08)--------------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 09)----CoalesceBatchesExec: target_batch_size=8192 10)------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 11)--------CoalesceBatchesExec: target_batch_size=8192 -12)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -13)------------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 +12)----------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 +13)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 14)--------------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 ######## @@ -773,8 +773,8 @@ physical_plan 12)------------CoalesceBatchesExec: target_batch_size=8192 13)--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 14)----------------CoalesceBatchesExec: target_batch_size=8192 -15)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -16)--------------------FilterExec: p_brand@1 = Brand#12 OR p_brand@1 = Brand#23, projection=[p_partkey@0] +15)------------------FilterExec: p_brand@1 = Brand#12 OR p_brand@1 = Brand#23, projection=[p_partkey@0] +16)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 17)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand], file_type=csv, has_header=true # Inlist simplification diff --git a/datafusion/sqllogictest/test_files/repartition.slt b/datafusion/sqllogictest/test_files/repartition.slt index 7041430155561..70666346e2cab 100644 --- a/datafusion/sqllogictest/test_files/repartition.slt +++ b/datafusion/sqllogictest/test_files/repartition.slt @@ -123,8 +123,8 @@ logical_plan physical_plan 01)CoalescePartitionsExec: fetch=5 02)--CoalesceBatchesExec: target_batch_size=8192, fetch=5 -03)----RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 -04)------FilterExec: c3@2 > 0 +03)----FilterExec: c3@2 > 0 +04)------RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 05)--------StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true # Start repratition on empty column test. diff --git a/datafusion/sqllogictest/test_files/select.slt b/datafusion/sqllogictest/test_files/select.slt index e16b81651c36f..aa14faf984e40 100644 --- a/datafusion/sqllogictest/test_files/select.slt +++ b/datafusion/sqllogictest/test_files/select.slt @@ -1441,8 +1441,8 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 -03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -04)------FilterExec: a@1 = 0 +03)----FilterExec: a@1 = 0 +04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true # source is ordered by a,b,c @@ -1462,8 +1462,8 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [c@3 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 -03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -04)------FilterExec: a@1 = 0 AND b@2 = 0 +03)----FilterExec: a@1 = 0 AND b@2 = 0 +04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true # source is ordered by a,b,c @@ -1483,8 +1483,8 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 -03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -04)------FilterExec: a@1 = 0 AND b@2 = 0 +03)----FilterExec: a@1 = 0 AND b@2 = 0 +04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true # source is ordered by a,b,c @@ -1504,8 +1504,8 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 -03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -04)------FilterExec: a@1 = 0 AND b@2 = 0 +03)----FilterExec: a@1 = 0 AND b@2 = 0 +04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true # source is ordered by a,b,c @@ -1526,8 +1526,8 @@ physical_plan 01)SortPreservingMergeExec: [c@3 ASC NULLS LAST] 02)--SortExec: expr=[c@3 ASC NULLS LAST], preserve_partitioning=[true] 03)----CoalesceBatchesExec: target_batch_size=8192 -04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)--------FilterExec: a@1 = 0 OR b@2 = 0 +04)------FilterExec: a@1 = 0 OR b@2 = 0 +05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true # When ordering lost during projection, we shouldn't keep the SortExec. @@ -1574,8 +1574,8 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [CAST(round(CAST(b@2 AS Float64)) AS Int32) ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 -03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -04)------FilterExec: CAST(round(CAST(b@2 AS Float64)) AS Int32) = a@1 +03)----FilterExec: CAST(round(CAST(b@2 AS Float64)) AS Int32) = a@1 +04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true diff --git a/datafusion/sqllogictest/test_files/simplify_expr.slt b/datafusion/sqllogictest/test_files/simplify_expr.slt index 9985ab49c2da0..075ccafcfd2e0 100644 --- a/datafusion/sqllogictest/test_files/simplify_expr.slt +++ b/datafusion/sqllogictest/test_files/simplify_expr.slt @@ -107,4 +107,3 @@ query B SELECT a / NULL::DECIMAL(4,3) > 1.2::decimal(2,1) FROM VALUES (1) AS t(a); ---- NULL - diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index c8401d2be604d..a0ac15b740d72 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -1190,13 +1190,12 @@ logical_plan 06)--------TableScan: t2 projection=[t2_id] physical_plan 01)CoalesceBatchesExec: target_batch_size=2 -02)--ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int] +02)--FilterExec: t1_id@0 > 40 OR NOT mark@3, projection=[t1_id@0, t1_name@1, t1_int@2] 03)----CoalesceBatchesExec: target_batch_size=2 -04)------FilterExec: t1_id@0 > 40 OR NOT mark@3 -05)--------HashJoinExec: mode=CollectLeft, join_type=LeftMark, on=[(t1_id@0, t2_id@0)] -06)----------DataSourceExec: partitions=1, partition_sizes=[1] -07)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)------------DataSourceExec: partitions=1, partition_sizes=[1] +04)------HashJoinExec: mode=CollectLeft, join_type=LeftMark, on=[(t1_id@0, t2_id@0)] +05)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +07)----------DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.explain.logical_plan_only = true; diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index cf286c8402358..356f1598bc0fa 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -516,8 +516,8 @@ physical_plan 10)------------------RepartitionExec: partitioning=Hash([c1@0], 4), input_partitions=4 11)--------------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[] 12)----------------------CoalesceBatchesExec: target_batch_size=2 -13)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -14)--------------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434, projection=[c1@0] +13)------------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434, projection=[c1@0] +14)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 15)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], file_type=csv, has_header=true 16)----ProjectionExec: expr=[1 as cnt] 17)------PlaceholderRowExec @@ -837,12 +837,12 @@ physical_plan 01)CoalescePartitionsExec 02)--UnionExec 03)----CoalesceBatchesExec: target_batch_size=2 -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------FilterExec: c1@0 = a +04)------FilterExec: c1@0 = a +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], file_type=csv, has_header=true 07)----CoalesceBatchesExec: target_batch_size=2 -08)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)--------FilterExec: c1@0 = a +08)------FilterExec: c1@0 = a +09)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 10)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], file_type=csv, has_header=true # Clean up after the test diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 8f7c1d21a3b3c..52cc80eae1c8a 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -1783,8 +1783,8 @@ physical_plan 08)--------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 09)----------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[] 10)------------------CoalesceBatchesExec: target_batch_size=4096 -11)--------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -12)----------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434, projection=[c1@0] +11)--------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434, projection=[c1@0] +12)----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 13)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], file_type=csv, has_header=true @@ -5205,8 +5205,8 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=1 06)----------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 07)------------CoalesceBatchesExec: target_batch_size=1 -08)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -09)----------------FilterExec: c1@0 = 2 OR c1@0 = 3 +08)--------------FilterExec: c1@0 = 2 OR c1@0 = 3 +09)----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 10)------------------DataSourceExec: partitions=1, partition_sizes=[1] query III @@ -5291,8 +5291,8 @@ physical_plan 07)------------CoalesceBatchesExec: target_batch_size=1 08)--------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 09)----------------CoalesceBatchesExec: target_batch_size=1 -10)------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -11)--------------------FilterExec: c1@0 = 1 +10)------------------FilterExec: c1@0 = 1 +11)--------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 12)----------------------DataSourceExec: partitions=1, partition_sizes=[1] query III @@ -5379,8 +5379,8 @@ physical_plan 10)------------------CoalesceBatchesExec: target_batch_size=1 11)--------------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 12)----------------------CoalesceBatchesExec: target_batch_size=1 -13)------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -14)--------------------------FilterExec: c1@0 > 1 +13)------------------------FilterExec: c1@0 > 1 +14)--------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 15)----------------------------DataSourceExec: partitions=1, partition_sizes=[1] query IIII From b30953fe108eba9c2b3c07b02cdedbdaa0b4e85e Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Thu, 17 Apr 2025 16:03:25 +0300 Subject: [PATCH 38/40] update other tests --- datafusion/core/tests/dataframe/mod.rs | 106 +++++++++---------- datafusion/core/tests/sql/explain_analyze.rs | 4 +- 2 files changed, 54 insertions(+), 56 deletions(-) diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index e4f7387afd136..1855a512048d6 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -2852,33 +2852,32 @@ async fn test_count_wildcard_on_where_scalar_subquery() -> Result<()> { assert_snapshot!( pretty_format_batches(&sql_results).unwrap(), @r" - +---------------+-----------------------------------------------------------------------------------------------------------------------------+ - | plan_type | plan | - +---------------+-----------------------------------------------------------------------------------------------------------------------------+ - | logical_plan | Projection: t1.a, t1.b | - | | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) | - | | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true | - | | Left Join: t1.a = __scalar_sq_1.a | - | | TableScan: t1 projection=[a, b] | - | | SubqueryAlias: __scalar_sq_1 | - | | Projection: count(Int64(1)) AS count(*), t2.a, Boolean(true) AS __always_true | - | | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1))]] | - | | TableScan: t2 projection=[a] | - | physical_plan | CoalesceBatchesExec: target_batch_size=8192 | - | | ProjectionExec: expr=[a@0 as a, b@1 as b] | - | | CoalesceBatchesExec: target_batch_size=8192 | - | | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0 | - | | HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, a@1)], projection=[a@0, b@1, count(*)@2, __always_true@4] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | ProjectionExec: expr=[count(Int64(1))@1 as count(*), a@0 as a, true as __always_true] | - | | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))] | - | | CoalesceBatchesExec: target_batch_size=8192 | - | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 | - | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | - | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | | - +---------------+-----------------------------------------------------------------------------------------------------------------------------+ + +---------------+---------------------------------------------------------------------------------------------------------------------------+ + | plan_type | plan | + +---------------+---------------------------------------------------------------------------------------------------------------------------+ + | logical_plan | Projection: t1.a, t1.b | + | | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) | + | | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true | + | | Left Join: t1.a = __scalar_sq_1.a | + | | TableScan: t1 projection=[a, b] | + | | SubqueryAlias: __scalar_sq_1 | + | | Projection: count(Int64(1)) AS count(*), t2.a, Boolean(true) AS __always_true | + | | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1))]] | + | | TableScan: t2 projection=[a] | + | physical_plan | CoalesceBatchesExec: target_batch_size=8192 | + | | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] | + | | CoalesceBatchesExec: target_batch_size=8192 | + | | HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, a@1)], projection=[a@0, b@1, count(*)@2, __always_true@4] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | ProjectionExec: expr=[count(Int64(1))@1 as count(*), a@0 as a, true as __always_true] | + | | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))] | + | | CoalesceBatchesExec: target_batch_size=8192 | + | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 | + | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | + | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | | + +---------------+---------------------------------------------------------------------------------------------------------------------------+ " ); @@ -2910,33 +2909,32 @@ async fn test_count_wildcard_on_where_scalar_subquery() -> Result<()> { assert_snapshot!( pretty_format_batches(&df_results).unwrap(), @r" - +---------------+-----------------------------------------------------------------------------------------------------------------------------+ - | plan_type | plan | - +---------------+-----------------------------------------------------------------------------------------------------------------------------+ - | logical_plan | Projection: t1.a, t1.b | - | | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) | - | | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true | - | | Left Join: t1.a = __scalar_sq_1.a | - | | TableScan: t1 projection=[a, b] | - | | SubqueryAlias: __scalar_sq_1 | - | | Projection: count(*), t2.a, Boolean(true) AS __always_true | - | | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1)) AS count(*)]] | - | | TableScan: t2 projection=[a] | - | physical_plan | CoalesceBatchesExec: target_batch_size=8192 | - | | ProjectionExec: expr=[a@0 as a, b@1 as b] | - | | CoalesceBatchesExec: target_batch_size=8192 | - | | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0 | - | | HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, a@1)], projection=[a@0, b@1, count(*)@2, __always_true@4] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | ProjectionExec: expr=[count(*)@1 as count(*), a@0 as a, true as __always_true] | - | | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(*)] | - | | CoalesceBatchesExec: target_batch_size=8192 | - | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 | - | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | - | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(*)] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | | - +---------------+-----------------------------------------------------------------------------------------------------------------------------+ + +---------------+---------------------------------------------------------------------------------------------------------------------------+ + | plan_type | plan | + +---------------+---------------------------------------------------------------------------------------------------------------------------+ + | logical_plan | Projection: t1.a, t1.b | + | | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) | + | | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true | + | | Left Join: t1.a = __scalar_sq_1.a | + | | TableScan: t1 projection=[a, b] | + | | SubqueryAlias: __scalar_sq_1 | + | | Projection: count(*), t2.a, Boolean(true) AS __always_true | + | | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1)) AS count(*)]] | + | | TableScan: t2 projection=[a] | + | physical_plan | CoalesceBatchesExec: target_batch_size=8192 | + | | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] | + | | CoalesceBatchesExec: target_batch_size=8192 | + | | HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, a@1)], projection=[a@0, b@1, count(*)@2, __always_true@4] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | ProjectionExec: expr=[count(*)@1 as count(*), a@0 as a, true as __always_true] | + | | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(*)] | + | | CoalesceBatchesExec: target_batch_size=8192 | + | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 | + | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | + | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(*)] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | | + +---------------+---------------------------------------------------------------------------------------------------------------------------+ " ); diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index cb71bd022bc6c..e8ef34c2afe70 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -609,8 +609,8 @@ async fn test_physical_plan_display_indent() { " RepartitionExec: partitioning=Hash([c1@0], 9000), input_partitions=9000", " AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[max(aggregate_test_100.c12), min(aggregate_test_100.c12)]", " CoalesceBatchesExec: target_batch_size=4096", - " RepartitionExec: partitioning=RoundRobinBatch(9000), input_partitions=1", - " FilterExec: c12@1 < 10", + " FilterExec: c12@1 < 10", + " RepartitionExec: partitioning=RoundRobinBatch(9000), input_partitions=1", " DataSourceExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1, c12], file_type=csv, has_header=true", ]; From ec54ccad92b6882674a87413b8509e09d96e0d64 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Thu, 17 Apr 2025 16:24:50 +0300 Subject: [PATCH 39/40] minor --- datafusion/datasource/src/file_scan_config.rs | 6 +++--- datafusion/datasource/src/source.rs | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index 4d300470a8976..761d2b655878f 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -607,7 +607,7 @@ impl DataSource for FileScanConfig { FilterPushdownSupport::Supported { child_descriptions, op, - revisit: retry, + revisit, } => { let new_data_source = Arc::new( FileScanConfigBuilder::from(self.clone()) @@ -616,13 +616,13 @@ impl DataSource for FileScanConfig { ); debug_assert!(child_descriptions.is_empty()); - debug_assert!(!retry); + debug_assert!(!revisit); Ok(FilterPushdownResult { support: FilterPushdownSupport::Supported { child_descriptions, op: new_data_source, - revisit: retry, + revisit, }, remaining_description, }) diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index 3de2719421cbf..2d6ea1a8b3915 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -220,18 +220,18 @@ impl ExecutionPlan for DataSourceExec { FilterPushdownSupport::Supported { child_descriptions, op, - revisit: retry, + revisit, } => { let new_exec = Arc::new(DataSourceExec::new(op)); debug_assert!(child_descriptions.is_empty()); - debug_assert!(!retry); + debug_assert!(!revisit); Ok(FilterPushdownResult { support: FilterPushdownSupport::Supported { child_descriptions, op: new_exec, - revisit: retry, + revisit, }, remaining_description, }) From 634531593469b92df3d992edba327812cd849eb6 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Thu, 17 Apr 2025 09:52:26 -0500 Subject: [PATCH 40/40] rename modules to match logical optimizer, tweak docs --- .../core/tests/physical_optimizer/mod.rs | 2 +- ...{filter_pushdown.rs => push_down_filter.rs} | 2 +- datafusion/physical-optimizer/src/lib.rs | 2 +- datafusion/physical-optimizer/src/optimizer.rs | 2 +- ...{filter_pushdown.rs => push_down_filter.rs} | 8 ++++---- datafusion/physical-plan/src/execution_plan.rs | 18 +++++++++++++++++- 6 files changed, 25 insertions(+), 9 deletions(-) rename datafusion/core/tests/physical_optimizer/{filter_pushdown.rs => push_down_filter.rs} (99%) rename datafusion/physical-optimizer/src/{filter_pushdown.rs => push_down_filter.rs} (98%) diff --git a/datafusion/core/tests/physical_optimizer/mod.rs b/datafusion/core/tests/physical_optimizer/mod.rs index fe7b9decfebfa..6643e7fd59b7a 100644 --- a/datafusion/core/tests/physical_optimizer/mod.rs +++ b/datafusion/core/tests/physical_optimizer/mod.rs @@ -21,11 +21,11 @@ mod aggregate_statistics; mod combine_partial_final_agg; mod enforce_distribution; mod enforce_sorting; -mod filter_pushdown; mod join_selection; mod limit_pushdown; mod limited_distinct_aggregation; mod projection_pushdown; +mod push_down_filter; mod replace_with_order_preserving_variants; mod sanity_checker; mod test_utils; diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs b/datafusion/core/tests/physical_optimizer/push_down_filter.rs similarity index 99% rename from datafusion/core/tests/physical_optimizer/filter_pushdown.rs rename to datafusion/core/tests/physical_optimizer/push_down_filter.rs index 4cef030e35cb6..b19144f1bcffe 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/push_down_filter.rs @@ -44,7 +44,7 @@ use datafusion_physical_expr::{ aggregate::AggregateExprBuilder, conjunction, Partitioning, }; use datafusion_physical_expr_common::physical_expr::fmt_sql; -use datafusion_physical_optimizer::filter_pushdown::PushdownFilter; +use datafusion_physical_optimizer::push_down_filter::PushdownFilter; use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::filter_pushdown::{ filter_pushdown_not_supported, FilterDescription, FilterPushdownResult, diff --git a/datafusion/physical-optimizer/src/lib.rs b/datafusion/physical-optimizer/src/lib.rs index 5a43d7118d638..57dac21b6eeed 100644 --- a/datafusion/physical-optimizer/src/lib.rs +++ b/datafusion/physical-optimizer/src/lib.rs @@ -29,7 +29,6 @@ pub mod coalesce_batches; pub mod combine_partial_final_agg; pub mod enforce_distribution; pub mod enforce_sorting; -pub mod filter_pushdown; pub mod join_selection; pub mod limit_pushdown; pub mod limited_distinct_aggregation; @@ -37,6 +36,7 @@ pub mod optimizer; pub mod output_requirements; pub mod projection_pushdown; pub mod pruning; +pub mod push_down_filter; pub mod sanity_checker; pub mod topk_aggregation; pub mod update_aggr_exprs; diff --git a/datafusion/physical-optimizer/src/optimizer.rs b/datafusion/physical-optimizer/src/optimizer.rs index 8bd22cbf1bdab..d4ff7d6b9e153 100644 --- a/datafusion/physical-optimizer/src/optimizer.rs +++ b/datafusion/physical-optimizer/src/optimizer.rs @@ -25,12 +25,12 @@ use crate::coalesce_batches::CoalesceBatches; use crate::combine_partial_final_agg::CombinePartialFinalAggregate; use crate::enforce_distribution::EnforceDistribution; use crate::enforce_sorting::EnforceSorting; -use crate::filter_pushdown::PushdownFilter; use crate::join_selection::JoinSelection; use crate::limit_pushdown::LimitPushdown; use crate::limited_distinct_aggregation::LimitedDistinctAggregation; use crate::output_requirements::OutputRequirements; use crate::projection_pushdown::ProjectionPushdown; +use crate::push_down_filter::PushdownFilter; use crate::sanity_checker::SanityCheckPlan; use crate::topk_aggregation::TopKAggregation; use crate::update_aggr_exprs::OptimizeAggregateOrder; diff --git a/datafusion/physical-optimizer/src/filter_pushdown.rs b/datafusion/physical-optimizer/src/push_down_filter.rs similarity index 98% rename from datafusion/physical-optimizer/src/filter_pushdown.rs rename to datafusion/physical-optimizer/src/push_down_filter.rs index c66f3278f6763..80201454d06d4 100644 --- a/datafusion/physical-optimizer/src/filter_pushdown.rs +++ b/datafusion/physical-optimizer/src/push_down_filter.rs @@ -238,8 +238,8 @@ use datafusion_physical_plan::ExecutionPlan; /// ``` /// /// The point here is that: -/// 1. We cannot push down `sum > 10` through the `AggregateExec` node into the `DataSourceExec` node. -/// Any filters above the `AggregateExec` node are not pushed down. +/// 1. We cannot push down `sum > 10` through the [`AggregateExec`] node into the `DataSourceExec` node. +/// Any filters above the [`AggregateExec`] node are not pushed down. /// This is determined by calling [`ExecutionPlan::try_pushdown_filters`] on the [`AggregateExec`] node. /// 2. We need to keep recursing into the tree so that we can discover the other [`FilterExec`] node and push /// down the `id=1` filter. @@ -307,8 +307,7 @@ use datafusion_physical_plan::ExecutionPlan; /// building a specialized [`PhysicalExpr`] that can be evaluated at runtime /// and internally maintains a reference to the hash table or other state. /// -/// To make working with these sorts of dynamic filters more tractable we have the method `PhysicalExpr::snapshot` -/// (TODO: add reference after is merged) +/// To make working with these sorts of dynamic filters more tractable we have the method [`PhysicalExpr::snapshot`] /// which attempts to simplify a dynamic filter into a "basic" non-dynamic filter. /// For a join this could mean converting it to an `InList` filter or a min/max filter for example. /// See `datafusion/physical-plan/src/dynamic_filters.rs` for more details. @@ -358,6 +357,7 @@ use datafusion_physical_plan::ExecutionPlan; /// /// /// [`PhysicalExpr`]: datafusion_physical_plan::PhysicalExpr +/// [`PhysicalExpr::snapshot`]: datafusion_physical_plan::PhysicalExpr::snapshot /// [`FilterExec`]: datafusion_physical_plan::filter::FilterExec /// [`ProjectionExec`]: datafusion_physical_plan::projection::ProjectionExec /// [`AggregateExec`]: datafusion_physical_plan::aggregates::AggregateExec diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index e508e8ad0fe80..2b6eac7be0675 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -473,15 +473,31 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// Attempts to recursively push given filters from the top of the tree into leafs. /// + /// This is used for various optimizations, such as: + /// + /// * Pushing down filters into scans in general to minimize the amount of data that needs to be materialzied. + /// * Pushing down dynamic filters from operators like TopK and Joins into scans. + /// + /// Generally the further down (closer to leaf nodes) that filters can be pushed, the better. + /// + /// Consider the case of a query such as `SELECT * FROM t WHERE a = 1 AND b = 2`. + /// With no filter pushdown the scan needs to read and materialize all the data from `t` and then filter based on `a` and `b`. + /// With filter pushdown into the scan it can first read only `a`, then `b` and keep track of + /// which rows match the filter. + /// Then only for rows that match the filter does it have to materialize the rest of the columns. + /// /// # Default Implementation /// /// The default implementation assumes: /// * Parent filters can't be passed onto children. /// * This node has no filters to contribute. /// + /// # Implementation Notes + /// + /// Most of the actual logic is implemented as a Physical Optimizer rule. /// See [`PushdownFilter`] for more details. /// - /// [`PushdownFilter`]: datafusion_physical_optimizer::filter_pushdown::PushdownFilter + /// [`PushdownFilter`]: https://docs.rs/datafusion/latest/datafusion/physical_optimizer/filter_pushdown/struct.PushdownFilter.html fn try_pushdown_filters( &self, fd: FilterDescription,