Skip to content

refactor filter pushdown apis #15801

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
merged 20 commits into from
May 5, 2025
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
4 changes: 4 additions & 0 deletions datafusion/core/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -179,6 +179,10 @@ name = "csv_load"
harness = false
name = "distinct_query_sql"

[[bench]]
harness = false
name = "push_down_filter"

[[bench]]
harness = false
name = "sort_limit_query_sql"
Expand Down
124 changes: 124 additions & 0 deletions datafusion/core/benches/push_down_filter.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,124 @@
// 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::array::RecordBatch;
use arrow::datatypes::{DataType, Field, Schema};
use bytes::{BufMut, BytesMut};
use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion};
use datafusion::config::ConfigOptions;
use datafusion::prelude::{ParquetReadOptions, SessionContext};
use datafusion_execution::object_store::ObjectStoreUrl;
use datafusion_physical_optimizer::filter_pushdown::FilterPushdown;
use datafusion_physical_optimizer::PhysicalOptimizerRule;
use datafusion_physical_plan::ExecutionPlan;
use object_store::memory::InMemory;
use object_store::path::Path;
use object_store::ObjectStore;
use parquet::arrow::ArrowWriter;
use std::sync::Arc;

async fn create_plan() -> Arc<dyn ExecutionPlan> {
let ctx = SessionContext::new();
let schema = Arc::new(Schema::new(vec![
Field::new("id", DataType::Int32, true),
Field::new("name", DataType::Utf8, true),
Field::new("age", DataType::UInt16, true),
Field::new("salary", DataType::Float64, true),
]));
let batch = RecordBatch::new_empty(schema);

let store = Arc::new(InMemory::new()) as Arc<dyn ObjectStore>;
let mut out = BytesMut::new().writer();
{
let mut writer = ArrowWriter::try_new(&mut out, batch.schema(), None).unwrap();
writer.write(&batch).unwrap();
writer.finish().unwrap();
}
let data = out.into_inner().freeze();
store
.put(&Path::from("test.parquet"), data.into())
.await
.unwrap();
ctx.register_object_store(
ObjectStoreUrl::parse("memory://").unwrap().as_ref(),
store,
);

ctx.register_parquet("t", "memory:///", ParquetReadOptions::default())
.await
.unwrap();

let df = ctx
.sql(
r"
WITH brackets AS (
SELECT age % 10 AS age_bracket
FROM t
GROUP BY age % 10
HAVING COUNT(*) > 10
)
SELECT id, name, age, salary
FROM t
JOIN brackets ON t.age % 10 = brackets.age_bracket
WHERE age > 20 AND t.salary > 1000
ORDER BY t.salary DESC
LIMIT 100
",
)
.await
.unwrap();

df.create_physical_plan().await.unwrap()
}

#[derive(Clone)]
struct BenchmarkPlan {
plan: Arc<dyn ExecutionPlan>,
config: ConfigOptions,
}

impl std::fmt::Display for BenchmarkPlan {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
write!(f, "BenchmarkPlan")
}
}

fn bench_push_down_filter(c: &mut Criterion) {
// Create a relatively complex plan
let plan = tokio::runtime::Runtime::new()
.unwrap()
.block_on(create_plan());
let mut config = ConfigOptions::default();
config.execution.parquet.pushdown_filters = true;
let plan = BenchmarkPlan { plan, config };

c.bench_with_input(
BenchmarkId::new("push_down_filter", plan.clone()),
&plan,
|b, plan| {
b.iter(|| {
let optimizer = FilterPushdown::new();
optimizer
.optimize(Arc::clone(&plan.plan), &plan.config)
.unwrap();
});
},
);
}

criterion_group!(benches, bench_push_down_filter);
criterion_main!(benches);
67 changes: 32 additions & 35 deletions datafusion/core/tests/physical_optimizer/push_down_filter.rs
Original file line number Diff line number Diff line change
Expand Up @@ -44,11 +44,10 @@ use datafusion_physical_expr::{
aggregate::AggregateExprBuilder, conjunction, Partitioning,
};
use datafusion_physical_expr_common::physical_expr::fmt_sql;
use datafusion_physical_optimizer::push_down_filter::PushdownFilter;
use datafusion_physical_optimizer::filter_pushdown::FilterPushdown;
use datafusion_physical_optimizer::PhysicalOptimizerRule;
use datafusion_physical_plan::filter_pushdown::{
filter_pushdown_not_supported, FilterDescription, FilterPushdownResult,
FilterPushdownSupport,
FilterPushdownPropagation, PredicateSupports,
};
use datafusion_physical_plan::{
aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy},
Expand Down Expand Up @@ -154,29 +153,24 @@ impl FileSource for TestSource {

fn try_pushdown_filters(
&self,
mut fd: FilterDescription,
mut filters: Vec<Arc<dyn PhysicalExpr>>,
config: &ConfigOptions,
) -> Result<FilterPushdownResult<Arc<dyn FileSource>>> {
) -> Result<FilterPushdownPropagation<Arc<dyn FileSource>>> {
if self.support && config.execution.parquet.pushdown_filters {
if let Some(internal) = self.predicate.as_ref() {
fd.filters.push(Arc::clone(internal));
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(all_filters)),
statistics: self.statistics.clone(), // should be updated in reality
}),
revisit: false,
},
remaining_description: FilterDescription::empty(),
let new_node = Arc::new(TestSource {
support: true,
predicate: Some(conjunction(filters.clone())),
statistics: self.statistics.clone(), // should be updated in reality
});
Ok(FilterPushdownPropagation {
filters: PredicateSupports::all_supported(filters),
updated_node: Some(new_node),
})
} else {
Ok(filter_pushdown_not_supported(fd))
Ok(FilterPushdownPropagation::unsupported(filters))
}
}
}
Expand All @@ -201,7 +195,7 @@ fn test_pushdown_into_scan() {

// expect the predicate to be pushed down into the DataSource
insta::assert_snapshot!(
OptimizationTest::new(plan, PushdownFilter{}, true),
OptimizationTest::new(plan, FilterPushdown{}, true),
@r"
OptimizationTest:
input:
Expand All @@ -225,7 +219,7 @@ fn test_pushdown_into_scan_with_config_options() {
insta::assert_snapshot!(
OptimizationTest::new(
Arc::clone(&plan),
PushdownFilter {},
FilterPushdown {},
false
),
@r"
Expand All @@ -244,7 +238,7 @@ fn test_pushdown_into_scan_with_config_options() {
insta::assert_snapshot!(
OptimizationTest::new(
plan,
PushdownFilter {},
FilterPushdown {},
true
),
@r"
Expand All @@ -269,7 +263,7 @@ fn test_filter_collapse() {
let plan = Arc::new(FilterExec::try_new(predicate2, filter1).unwrap());

insta::assert_snapshot!(
OptimizationTest::new(plan, PushdownFilter{}, true),
OptimizationTest::new(plan, FilterPushdown{}, true),
@r"
OptimizationTest:
input:
Expand All @@ -278,7 +272,7 @@ fn test_filter_collapse() {
- 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, pushdown_supported=true, predicate=b@1 = bar AND a@0 = foo
- DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=a@0 = foo AND b@1 = bar
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This actually looks like an improvement to me as now a = foo will be evaluated before b=bar as was done in the input plan. This might be important for short circuiting, perhaps

The prior version of this optimization seems to have reordered them

"
);
}
Expand All @@ -288,25 +282,28 @@ fn test_filter_with_projection() {
let scan = test_scan(true);
let projection = vec![1, 0];
let predicate = col_lit_predicate("a", "foo", schema());
let plan = Arc::new(
FilterExec::try_new(predicate, Arc::clone(&scan))
let filter = Arc::new(
FilterExec::try_new(Arc::clone(&predicate), Arc::clone(&scan))
.unwrap()
.with_projection(Some(projection))
.unwrap(),
);
let predicate = col_lit_predicate("b", "bar", &filter.schema());
let plan = Arc::new(FilterExec::try_new(predicate, filter).unwrap());

// 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{}, true),
OptimizationTest::new(plan, FilterPushdown{}, true),
@r"
OptimizationTest:
input:
- FilterExec: a@0 = foo, projection=[b@1, a@0]
- DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, pushdown_supported=true
- FilterExec: b@0 = bar
- 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, pushdown_supported=true, predicate=a@0 = foo
- DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=a@0 = foo AND b@1 = bar
",
);

Expand All @@ -320,7 +317,7 @@ fn test_filter_with_projection() {
.unwrap(),
);
insta::assert_snapshot!(
OptimizationTest::new(plan, PushdownFilter{},true),
OptimizationTest::new(plan, FilterPushdown{},true),
@r"
OptimizationTest:
input:
Expand Down Expand Up @@ -349,7 +346,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{},true),
OptimizationTest::new(plan, FilterPushdown{},true),
@r"
OptimizationTest:
input:
Expand All @@ -362,7 +359,7 @@ fn test_push_down_through_transparent_nodes() {
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, pushdown_supported=true, predicate=b@1 = bar AND a@0 = foo
- DataSourceExec: file_groups={0 groups: []}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=a@0 = foo AND b@1 = bar
"
);
}
Expand Down Expand Up @@ -413,7 +410,7 @@ fn test_no_pushdown_through_aggregates() {

// expect the predicate to be pushed down into the DataSource
insta::assert_snapshot!(
OptimizationTest::new(plan, PushdownFilter{}, true),
OptimizationTest::new(plan, FilterPushdown{}, true),
@r"
OptimizationTest:
input:
Expand Down
16 changes: 7 additions & 9 deletions datafusion/datasource/src/file.rs
Original file line number Diff line number Diff line change
Expand Up @@ -28,10 +28,8 @@ use crate::file_stream::FileOpener;
use arrow::datatypes::SchemaRef;
use datafusion_common::config::ConfigOptions;
use datafusion_common::{Result, Statistics};
use datafusion_physical_expr::LexOrdering;
use datafusion_physical_plan::filter_pushdown::{
filter_pushdown_not_supported, FilterDescription, FilterPushdownResult,
};
use datafusion_physical_expr::{LexOrdering, PhysicalExpr};
use datafusion_physical_plan::filter_pushdown::FilterPushdownPropagation;
use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet;
use datafusion_physical_plan::DisplayFormatType;

Expand Down Expand Up @@ -108,14 +106,14 @@ pub trait FileSource: Send + Sync {
}

/// Try to push down filters into this FileSource.
/// See [`ExecutionPlan::try_pushdown_filters`] for more details.
/// See [`ExecutionPlan::handle_child_pushdown_result`] for more details.
///
/// [`ExecutionPlan::try_pushdown_filters`]: datafusion_physical_plan::ExecutionPlan::try_pushdown_filters
/// [`ExecutionPlan::handle_child_pushdown_result`]: datafusion_physical_plan::ExecutionPlan::handle_child_pushdown_result
fn try_pushdown_filters(
&self,
fd: FilterDescription,
filters: Vec<Arc<dyn PhysicalExpr>>,
_config: &ConfigOptions,
) -> Result<FilterPushdownResult<Arc<dyn FileSource>>> {
Ok(filter_pushdown_not_supported(fd))
) -> Result<FilterPushdownPropagation<Arc<dyn FileSource>>> {
Ok(FilterPushdownPropagation::unsupported(filters))
}
}
Loading