Skip to content

Commit be42f3d

Browse files
itsjunetimealamb
andauthored
Return TableProviderFilterPushDown::Exact when Parquet Pushdown Enabled (#12135)
* feat: Preemptively filter for pushdown-preventing columns in ListingTable * Fix behavior to make all previous tests work and lay groundwork for future tests * fix: Add some more tests and fix small issue with pushdown specificity * test: Revive unneccesarily removed test * ci: Fix CI issues with different combinations of exprs * fix: run fmt * Fix doc publicity issues * Add ::new fn for PushdownChecker * Remove unnecessary 'pub' qualifier * Fix naming and doc comment of non_pushdown_columns to reflect what it actually does (the opposite) and add back useful comments * fmt * Extend FileFormat trait to allow library users to define formats which support pushdown * fmt * fix: reference real fn in doc to fix CI * Minor: Add tests for using FilterExec when parquet was pushed down * Update datafusion/core/src/datasource/file_format/mod.rs * Pipe schema information through to TableScan and ParquetExec to facilitate unnecessary FilterExec removal * - Remove collect::<(_, _)> to satisfy msrv - Remove expect(_) attr to satisfy msrv - Update comments with more accurate details and explanations * Add more details in comments for `map_partial_batch` Co-authored-by: Andrew Lamb <[email protected]> * Remove reference to issue #4028 as it will be closed * Convert normal comments to doc-comments Co-authored-by: Andrew Lamb <[email protected]> * Clarify meaning of word `projected` in comment Co-authored-by: Andrew Lamb <[email protected]> * Clarify more how `table_schema` is used differently from `projected_table_schema` Co-authored-by: Andrew Lamb <[email protected]> * Finish partially-written comment about SchemaMapping struct --------- Co-authored-by: Andrew Lamb <[email protected]>
1 parent b309525 commit be42f3d

File tree

25 files changed

+662
-297
lines changed

25 files changed

+662
-297
lines changed

datafusion/core/src/datasource/file_format/mod.rs

Lines changed: 28 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -45,6 +45,7 @@ use crate::physical_plan::{ExecutionPlan, Statistics};
4545
use arrow_schema::{DataType, Field, Schema};
4646
use datafusion_common::file_options::file_type::FileType;
4747
use datafusion_common::{internal_err, not_impl_err, GetExt};
48+
use datafusion_expr::Expr;
4849
use datafusion_physical_expr::PhysicalExpr;
4950

5051
use async_trait::async_trait;
@@ -138,6 +139,33 @@ pub trait FileFormat: Send + Sync + fmt::Debug {
138139
) -> Result<Arc<dyn ExecutionPlan>> {
139140
not_impl_err!("Writer not implemented for this format")
140141
}
142+
143+
/// Check if the specified file format has support for pushing down the provided filters within
144+
/// the given schemas. Added initially to support the Parquet file format's ability to do this.
145+
fn supports_filters_pushdown(
146+
&self,
147+
_file_schema: &Schema,
148+
_table_schema: &Schema,
149+
_filters: &[&Expr],
150+
) -> Result<FilePushdownSupport> {
151+
Ok(FilePushdownSupport::NoSupport)
152+
}
153+
}
154+
155+
/// An enum to distinguish between different states when determining if certain filters can be
156+
/// pushed down to file scanning
157+
#[derive(Debug, PartialEq)]
158+
pub enum FilePushdownSupport {
159+
/// The file format/system being asked does not support any sort of pushdown. This should be
160+
/// used even if the file format theoretically supports some sort of pushdown, but it's not
161+
/// enabled or implemented yet.
162+
NoSupport,
163+
/// The file format/system being asked *does* support pushdown, but it can't make it work for
164+
/// the provided filter/expression
165+
NotSupportedForFilter,
166+
/// The file format/system being asked *does* support pushdown and *can* make it work for the
167+
/// provided filter/expression
168+
Supported,
141169
}
142170

143171
/// A container of [FileFormatFactory] which also implements [FileType].

datafusion/core/src/datasource/file_format/parquet.rs

Lines changed: 26 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -26,7 +26,7 @@ use super::write::demux::start_demuxer_task;
2626
use super::write::{create_writer, SharedBuffer};
2727
use super::{
2828
coerce_file_schema_to_view_type, transform_schema_to_view, FileFormat,
29-
FileFormatFactory, FileScanConfig,
29+
FileFormatFactory, FilePushdownSupport, FileScanConfig,
3030
};
3131
use crate::arrow::array::RecordBatch;
3232
use crate::arrow::datatypes::{Fields, Schema, SchemaRef};
@@ -53,6 +53,7 @@ use datafusion_common::{
5353
use datafusion_common_runtime::SpawnedTask;
5454
use datafusion_execution::memory_pool::{MemoryConsumer, MemoryPool, MemoryReservation};
5555
use datafusion_execution::TaskContext;
56+
use datafusion_expr::Expr;
5657
use datafusion_functions_aggregate::min_max::{MaxAccumulator, MinAccumulator};
5758
use datafusion_physical_expr::PhysicalExpr;
5859
use datafusion_physical_plan::metrics::MetricsSet;
@@ -78,7 +79,9 @@ use tokio::io::{AsyncWrite, AsyncWriteExt};
7879
use tokio::sync::mpsc::{self, Receiver, Sender};
7980
use tokio::task::JoinSet;
8081

81-
use crate::datasource::physical_plan::parquet::ParquetExecBuilder;
82+
use crate::datasource::physical_plan::parquet::{
83+
can_expr_be_pushed_down_with_schemas, ParquetExecBuilder,
84+
};
8285
use datafusion_physical_expr_common::sort_expr::LexRequirement;
8386
use futures::{StreamExt, TryStreamExt};
8487
use object_store::path::Path;
@@ -414,6 +417,27 @@ impl FileFormat for ParquetFormat {
414417
order_requirements,
415418
)) as _)
416419
}
420+
421+
fn supports_filters_pushdown(
422+
&self,
423+
file_schema: &Schema,
424+
table_schema: &Schema,
425+
filters: &[&Expr],
426+
) -> Result<FilePushdownSupport> {
427+
if !self.options().global.pushdown_filters {
428+
return Ok(FilePushdownSupport::NoSupport);
429+
}
430+
431+
let all_supported = filters.iter().all(|filter| {
432+
can_expr_be_pushed_down_with_schemas(filter, file_schema, table_schema)
433+
});
434+
435+
Ok(if all_supported {
436+
FilePushdownSupport::Supported
437+
} else {
438+
FilePushdownSupport::NotSupportedForFilter
439+
})
440+
}
417441
}
418442

419443
/// Fetches parquet metadata from ObjectStore for given object

datafusion/core/src/datasource/listing/helpers.rs

Lines changed: 54 additions & 56 deletions
Original file line numberDiff line numberDiff line change
@@ -53,66 +53,64 @@ use object_store::{ObjectMeta, ObjectStore};
5353
/// was performed
5454
pub fn expr_applicable_for_cols(col_names: &[&str], expr: &Expr) -> bool {
5555
let mut is_applicable = true;
56-
expr.apply(|expr| {
57-
match expr {
58-
Expr::Column(Column { ref name, .. }) => {
59-
is_applicable &= col_names.contains(&name.as_str());
60-
if is_applicable {
61-
Ok(TreeNodeRecursion::Jump)
62-
} else {
63-
Ok(TreeNodeRecursion::Stop)
64-
}
56+
expr.apply(|expr| match expr {
57+
Expr::Column(Column { ref name, .. }) => {
58+
is_applicable &= col_names.contains(&name.as_str());
59+
if is_applicable {
60+
Ok(TreeNodeRecursion::Jump)
61+
} else {
62+
Ok(TreeNodeRecursion::Stop)
6563
}
66-
Expr::Literal(_)
67-
| Expr::Alias(_)
68-
| Expr::OuterReferenceColumn(_, _)
69-
| Expr::ScalarVariable(_, _)
70-
| Expr::Not(_)
71-
| Expr::IsNotNull(_)
72-
| Expr::IsNull(_)
73-
| Expr::IsTrue(_)
74-
| Expr::IsFalse(_)
75-
| Expr::IsUnknown(_)
76-
| Expr::IsNotTrue(_)
77-
| Expr::IsNotFalse(_)
78-
| Expr::IsNotUnknown(_)
79-
| Expr::Negative(_)
80-
| Expr::Cast { .. }
81-
| Expr::TryCast { .. }
82-
| Expr::BinaryExpr { .. }
83-
| Expr::Between { .. }
84-
| Expr::Like { .. }
85-
| Expr::SimilarTo { .. }
86-
| Expr::InList { .. }
87-
| Expr::Exists { .. }
88-
| Expr::InSubquery(_)
89-
| Expr::ScalarSubquery(_)
90-
| Expr::GroupingSet(_)
91-
| Expr::Case { .. } => Ok(TreeNodeRecursion::Continue),
92-
93-
Expr::ScalarFunction(scalar_function) => {
94-
match scalar_function.func.signature().volatility {
95-
Volatility::Immutable => Ok(TreeNodeRecursion::Continue),
96-
// TODO: Stable functions could be `applicable`, but that would require access to the context
97-
Volatility::Stable | Volatility::Volatile => {
98-
is_applicable = false;
99-
Ok(TreeNodeRecursion::Stop)
100-
}
64+
}
65+
Expr::Literal(_)
66+
| Expr::Alias(_)
67+
| Expr::OuterReferenceColumn(_, _)
68+
| Expr::ScalarVariable(_, _)
69+
| Expr::Not(_)
70+
| Expr::IsNotNull(_)
71+
| Expr::IsNull(_)
72+
| Expr::IsTrue(_)
73+
| Expr::IsFalse(_)
74+
| Expr::IsUnknown(_)
75+
| Expr::IsNotTrue(_)
76+
| Expr::IsNotFalse(_)
77+
| Expr::IsNotUnknown(_)
78+
| Expr::Negative(_)
79+
| Expr::Cast(_)
80+
| Expr::TryCast(_)
81+
| Expr::BinaryExpr(_)
82+
| Expr::Between(_)
83+
| Expr::Like(_)
84+
| Expr::SimilarTo(_)
85+
| Expr::InList(_)
86+
| Expr::Exists(_)
87+
| Expr::InSubquery(_)
88+
| Expr::ScalarSubquery(_)
89+
| Expr::GroupingSet(_)
90+
| Expr::Case(_) => Ok(TreeNodeRecursion::Continue),
91+
92+
Expr::ScalarFunction(scalar_function) => {
93+
match scalar_function.func.signature().volatility {
94+
Volatility::Immutable => Ok(TreeNodeRecursion::Continue),
95+
// TODO: Stable functions could be `applicable`, but that would require access to the context
96+
Volatility::Stable | Volatility::Volatile => {
97+
is_applicable = false;
98+
Ok(TreeNodeRecursion::Stop)
10199
}
102100
}
101+
}
103102

104-
// TODO other expressions are not handled yet:
105-
// - AGGREGATE and WINDOW should not end up in filter conditions, except maybe in some edge cases
106-
// - Can `Wildcard` be considered as a `Literal`?
107-
// - ScalarVariable could be `applicable`, but that would require access to the context
108-
Expr::AggregateFunction { .. }
109-
| Expr::WindowFunction { .. }
110-
| Expr::Wildcard { .. }
111-
| Expr::Unnest { .. }
112-
| Expr::Placeholder(_) => {
113-
is_applicable = false;
114-
Ok(TreeNodeRecursion::Stop)
115-
}
103+
// TODO other expressions are not handled yet:
104+
// - AGGREGATE and WINDOW should not end up in filter conditions, except maybe in some edge cases
105+
// - Can `Wildcard` be considered as a `Literal`?
106+
// - ScalarVariable could be `applicable`, but that would require access to the context
107+
Expr::AggregateFunction { .. }
108+
| Expr::WindowFunction { .. }
109+
| Expr::Wildcard { .. }
110+
| Expr::Unnest { .. }
111+
| Expr::Placeholder(_) => {
112+
is_applicable = false;
113+
Ok(TreeNodeRecursion::Stop)
116114
}
117115
})
118116
.unwrap();

datafusion/core/src/datasource/listing/table.rs

Lines changed: 43 additions & 28 deletions
Original file line numberDiff line numberDiff line change
@@ -18,16 +18,17 @@
1818
//! The table implementation.
1919
2020
use std::collections::HashMap;
21-
use std::str::FromStr;
22-
use std::{any::Any, sync::Arc};
21+
use std::{any::Any, str::FromStr, sync::Arc};
2322

2423
use super::helpers::{expr_applicable_for_cols, pruned_partition_list, split_files};
25-
use super::PartitionedFile;
24+
use super::{ListingTableUrl, PartitionedFile};
2625

27-
use super::ListingTableUrl;
28-
use crate::datasource::{create_ordering, get_statistics_with_limit};
2926
use crate::datasource::{
30-
file_format::{file_compression_type::FileCompressionType, FileFormat},
27+
create_ordering,
28+
file_format::{
29+
file_compression_type::FileCompressionType, FileFormat, FilePushdownSupport,
30+
},
31+
get_statistics_with_limit,
3132
physical_plan::{FileScanConfig, FileSinkConfig},
3233
};
3334
use crate::execution::context::SessionState;
@@ -43,8 +44,9 @@ use datafusion_common::{
4344
config_datafusion_err, internal_err, plan_err, project_schema, Constraints,
4445
SchemaExt, ToDFSchema,
4546
};
46-
use datafusion_execution::cache::cache_manager::FileStatisticsCache;
47-
use datafusion_execution::cache::cache_unit::DefaultFileStatisticsCache;
47+
use datafusion_execution::cache::{
48+
cache_manager::FileStatisticsCache, cache_unit::DefaultFileStatisticsCache,
49+
};
4850
use datafusion_physical_expr::{
4951
create_physical_expr, LexOrdering, PhysicalSortRequirement,
5052
};
@@ -817,19 +819,22 @@ impl TableProvider for ListingTable {
817819
.map(|col| Ok(self.table_schema.field_with_name(&col.0)?.clone()))
818820
.collect::<Result<Vec<_>>>()?;
819821

820-
let filters = if let Some(expr) = conjunction(filters.to_vec()) {
821-
// NOTE: Use the table schema (NOT file schema) here because `expr` may contain references to partition columns.
822-
let table_df_schema = self.table_schema.as_ref().clone().to_dfschema()?;
823-
let filters =
824-
create_physical_expr(&expr, &table_df_schema, state.execution_props())?;
825-
Some(filters)
826-
} else {
827-
None
828-
};
822+
let filters = conjunction(filters.to_vec())
823+
.map(|expr| -> Result<_> {
824+
// NOTE: Use the table schema (NOT file schema) here because `expr` may contain references to partition columns.
825+
let table_df_schema = self.table_schema.as_ref().clone().to_dfschema()?;
826+
let filters = create_physical_expr(
827+
&expr,
828+
&table_df_schema,
829+
state.execution_props(),
830+
)?;
831+
Ok(Some(filters))
832+
})
833+
.unwrap_or(Ok(None))?;
829834

830-
let object_store_url = if let Some(url) = self.table_paths.first() {
831-
url.object_store()
832-
} else {
835+
let Some(object_store_url) =
836+
self.table_paths.first().map(ListingTableUrl::object_store)
837+
else {
833838
return Ok(Arc::new(EmptyExec::new(Arc::new(Schema::empty()))));
834839
};
835840

@@ -854,27 +859,37 @@ impl TableProvider for ListingTable {
854859
&self,
855860
filters: &[&Expr],
856861
) -> Result<Vec<TableProviderFilterPushDown>> {
857-
Ok(filters
862+
filters
858863
.iter()
859864
.map(|filter| {
860865
if expr_applicable_for_cols(
861866
&self
862867
.options
863868
.table_partition_cols
864869
.iter()
865-
.map(|x| x.0.as_str())
870+
.map(|col| col.0.as_str())
866871
.collect::<Vec<_>>(),
867872
filter,
868873
) {
869874
// if filter can be handled by partition pruning, it is exact
870-
TableProviderFilterPushDown::Exact
871-
} else {
872-
// otherwise, we still might be able to handle the filter with file
873-
// level mechanisms such as Parquet row group pruning.
874-
TableProviderFilterPushDown::Inexact
875+
return Ok(TableProviderFilterPushDown::Exact);
876+
}
877+
878+
// if we can't push it down completely with only the filename-based/path-based
879+
// column names, then we should check if we can do parquet predicate pushdown
880+
let supports_pushdown = self.options.format.supports_filters_pushdown(
881+
&self.file_schema,
882+
&self.table_schema,
883+
&[filter],
884+
)?;
885+
886+
if supports_pushdown == FilePushdownSupport::Supported {
887+
return Ok(TableProviderFilterPushDown::Exact);
875888
}
889+
890+
Ok(TableProviderFilterPushDown::Inexact)
876891
})
877-
.collect())
892+
.collect()
878893
}
879894

880895
fn get_table_definition(&self) -> Option<&str> {

datafusion/core/src/datasource/physical_plan/file_scan_config.rs

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -258,7 +258,7 @@ impl FileScanConfig {
258258
(projected_schema, table_stats, projected_output_ordering)
259259
}
260260

261-
#[allow(unused)] // Only used by avro
261+
#[cfg_attr(not(feature = "avro"), allow(unused))] // Only used by avro
262262
pub(crate) fn projected_file_column_names(&self) -> Option<Vec<String>> {
263263
self.projection.as_ref().map(|p| {
264264
p.iter()

datafusion/core/src/datasource/physical_plan/mod.rs

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -516,7 +516,8 @@ mod tests {
516516
Field::new("c3", DataType::Float64, true),
517517
]));
518518

519-
let adapter = DefaultSchemaAdapterFactory::default().create(table_schema.clone());
519+
let adapter = DefaultSchemaAdapterFactory
520+
.create(table_schema.clone(), table_schema.clone());
520521

521522
let file_schema = Schema::new(vec![
522523
Field::new("c1", DataType::Utf8, true),
@@ -573,7 +574,7 @@ mod tests {
573574

574575
let indices = vec![1, 2, 4];
575576
let schema = SchemaRef::from(table_schema.project(&indices).unwrap());
576-
let adapter = DefaultSchemaAdapterFactory::default().create(schema);
577+
let adapter = DefaultSchemaAdapterFactory.create(schema, table_schema.clone());
577578
let (mapping, projection) = adapter.map_schema(&file_schema).unwrap();
578579

579580
let id = Int32Array::from(vec![Some(1), Some(2), Some(3)]);

datafusion/core/src/datasource/physical_plan/parquet/mod.rs

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -61,6 +61,7 @@ pub use access_plan::{ParquetAccessPlan, RowGroupAccess};
6161
pub use metrics::ParquetFileMetrics;
6262
use opener::ParquetOpener;
6363
pub use reader::{DefaultParquetFileReaderFactory, ParquetFileReaderFactory};
64+
pub use row_filter::can_expr_be_pushed_down_with_schemas;
6465
pub use writer::plan_to_parquet;
6566

6667
/// Execution plan for reading one or more Parquet files.
@@ -405,6 +406,7 @@ impl ParquetExecBuilder {
405406

406407
let (projected_schema, projected_statistics, projected_output_ordering) =
407408
base_config.project();
409+
408410
let cache = ParquetExec::compute_properties(
409411
projected_schema,
410412
&projected_output_ordering,
@@ -707,7 +709,7 @@ impl ExecutionPlan for ParquetExec {
707709
let schema_adapter_factory = self
708710
.schema_adapter_factory
709711
.clone()
710-
.unwrap_or_else(|| Arc::new(DefaultSchemaAdapterFactory::default()));
712+
.unwrap_or_else(|| Arc::new(DefaultSchemaAdapterFactory));
711713

712714
let opener = ParquetOpener {
713715
partition_index,

datafusion/core/src/datasource/physical_plan/parquet/opener.rs

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -99,7 +99,9 @@ impl FileOpener for ParquetOpener {
9999

100100
let projected_schema =
101101
SchemaRef::from(self.table_schema.project(&self.projection)?);
102-
let schema_adapter = self.schema_adapter_factory.create(projected_schema);
102+
let schema_adapter = self
103+
.schema_adapter_factory
104+
.create(projected_schema, self.table_schema.clone());
103105
let predicate = self.predicate.clone();
104106
let pruning_predicate = self.pruning_predicate.clone();
105107
let page_pruning_predicate = self.page_pruning_predicate.clone();

0 commit comments

Comments
 (0)