Skip to content

Implement Parquet filter pushdown via new filter pushdown APIs #15769

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

Open
wants to merge 15 commits into
base: main
Choose a base branch
from
2 changes: 1 addition & 1 deletion datafusion-examples/examples/advanced_parquet_index.rs
Original file line number Diff line number Diff line change
Expand Up @@ -495,7 +495,7 @@ impl TableProvider for IndexTableProvider {
ParquetSource::default()
// provide the predicate so the DataSourceExec can try and prune
// row groups internally
.with_predicate(Arc::clone(&schema), predicate)
.with_predicate(predicate)
// provide the factory to create parquet reader without re-reading metadata
.with_parquet_file_reader_factory(Arc::new(reader_factory)),
);
Expand Down
3 changes: 1 addition & 2 deletions datafusion-examples/examples/parquet_index.rs
Original file line number Diff line number Diff line change
Expand Up @@ -242,8 +242,7 @@ impl TableProvider for IndexTableProvider {
let files = self.index.get_files(predicate.clone())?;

let object_store_url = ObjectStoreUrl::parse("file://")?;
let source =
Arc::new(ParquetSource::default().with_predicate(self.schema(), predicate));
let source = Arc::new(ParquetSource::default().with_predicate(predicate));
let mut file_scan_config_builder =
FileScanConfigBuilder::new(object_store_url, self.schema(), source)
.with_projection(projection.cloned())
Expand Down
16 changes: 1 addition & 15 deletions datafusion/core/src/datasource/listing/table.rs
Original file line number Diff line number Diff line change
Expand Up @@ -24,9 +24,7 @@ use std::{any::Any, str::FromStr, sync::Arc};

use crate::datasource::{
create_ordering,
file_format::{
file_compression_type::FileCompressionType, FileFormat, FilePushdownSupport,
},
file_format::{file_compression_type::FileCompressionType, FileFormat},
physical_plan::FileSinkConfig,
};
use crate::execution::context::SessionState;
Expand Down Expand Up @@ -982,18 +980,6 @@ impl TableProvider for ListingTable {
return Ok(TableProviderFilterPushDown::Exact);
}

// if we can't push it down completely with only the filename-based/path-based
// column names, then we should check if we can do parquet predicate pushdown
let supports_pushdown = self.options.format.supports_filters_pushdown(
&self.file_schema,
&self.table_schema,
&[filter],
)?;

if supports_pushdown == FilePushdownSupport::Supported {
return Ok(TableProviderFilterPushDown::Exact);
}
Comment on lines -985 to -995
Copy link
Contributor Author

Choose a reason for hiding this comment

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

The point of this PR is that this moves from being something specialized that ListingTable does to anything that works for any TableProvider / they don't need to do anything special! The checks for compatibility also happen all within the parquet data source machinery, instead of leaking implementations via supports_filters_pushdown.

Copy link
Contributor

Choose a reason for hiding this comment

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

I have one question: aren't we expecting/preparing for, people to use ListingTable if they read Parquet files? Are we eventually planning to remove all format-specific handlings? Or this is a case only for filter pushdown?

Copy link
Contributor

Choose a reason for hiding this comment

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

If that's the case, why don't we fully remove supports_filters_pushdown() API at all

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I think many users of DataFusion (based on our usage, talks I've seen and examples we have) use custom TableProvider implementations.

I would keep supports_filters_pushdown so that TableProviders can do Exact pruning of filters, e.g. using partition columns.

Copy link
Contributor

Choose a reason for hiding this comment

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

We can justify implementing other TableProviders for Parquet, but still I cannot understand why we need to degrade the capabilities of our ListingTable. Is't it always better pruning/simplifying things at the higher levels as possible?


Ok(TableProviderFilterPushDown::Inexact)
})
.collect()
Expand Down
9 changes: 5 additions & 4 deletions datafusion/core/src/datasource/physical_plan/parquet.rs
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,7 @@ mod tests {
use datafusion_datasource::file_scan_config::FileScanConfigBuilder;
use datafusion_datasource::source::DataSourceExec;

use datafusion_datasource::file::FileSource;
use datafusion_datasource::{FileRange, PartitionedFile};
use datafusion_datasource_parquet::source::ParquetSource;
use datafusion_datasource_parquet::{
Expand Down Expand Up @@ -139,7 +140,7 @@ mod tests {
self.round_trip(batches).await.batches
}

fn build_file_source(&self, file_schema: SchemaRef) -> Arc<ParquetSource> {
fn build_file_source(&self, file_schema: SchemaRef) -> Arc<dyn FileSource> {
// set up predicate (this is normally done by a layer higher up)
let predicate = self
.predicate
Expand All @@ -148,7 +149,7 @@ mod tests {

let mut source = ParquetSource::default();
if let Some(predicate) = predicate {
source = source.with_predicate(Arc::clone(&file_schema), predicate);
source = source.with_predicate(predicate);
Comment on lines -151 to +152
Copy link
Contributor Author

Choose a reason for hiding this comment

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

This seemed like an easy win since I was able to just change this so that the schema is always passed in by the FileSourceConfigBuilder instead of only when with_predicate is called.
This was necessary becasue with_predicate is no longer called to attach a predicate, instaed it happens during an optimization pass so ParquetSource neesd to have it available at that point.
I left with_predicate in there to avoid churn and in case there is a use case for attaching a predicate directly through the scan instad of a as a FilterExec that later gets pushed into the scan.

}

if self.pushdown_predicate {
Expand All @@ -161,14 +162,14 @@ mod tests {
source = source.with_enable_page_index(true);
}

Arc::new(source)
source.with_schema(Arc::clone(&file_schema))
}

fn build_parquet_exec(
&self,
file_schema: SchemaRef,
file_group: FileGroup,
source: Arc<ParquetSource>,
source: Arc<dyn FileSource>,
) -> Arc<DataSourceExec> {
let base_config = FileScanConfigBuilder::new(
ObjectStoreUrl::local_filesystem(),
Expand Down
10 changes: 6 additions & 4 deletions datafusion/core/src/test_util/parquet.rs
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@ use crate::physical_plan::metrics::MetricsSet;
use crate::physical_plan::ExecutionPlan;
use crate::prelude::{Expr, SessionConfig, SessionContext};

use datafusion_datasource::file::FileSource;
use datafusion_datasource::file_scan_config::FileScanConfigBuilder;
use datafusion_datasource::source::DataSourceExec;
use object_store::path::Path;
Expand Down Expand Up @@ -182,10 +183,11 @@ impl TestParquetFile {
let physical_filter_expr =
create_physical_expr(&filter, &df_schema, &ExecutionProps::default())?;

let source = Arc::new(ParquetSource::new(parquet_options).with_predicate(
Arc::clone(&self.schema),
Arc::clone(&physical_filter_expr),
));
let source = Arc::new(
ParquetSource::new(parquet_options)
.with_predicate(Arc::clone(&physical_filter_expr)),
)
.with_schema(Arc::clone(&self.schema));
let config = scan_config_builder.with_source(source).build();
let parquet_exec = DataSourceExec::from_data_source(config);

Expand Down
2 changes: 1 addition & 1 deletion datafusion/core/tests/fuzz_cases/pruning.rs
Original file line number Diff line number Diff line change
Expand Up @@ -276,7 +276,7 @@ async fn execute_with_predicate(
ctx: &SessionContext,
) -> Vec<String> {
let parquet_source = if prune_stats {
ParquetSource::default().with_predicate(Arc::clone(&schema), predicate.clone())
ParquetSource::default().with_predicate(predicate.clone())
} else {
ParquetSource::default()
};
Expand Down
2 changes: 1 addition & 1 deletion datafusion/core/tests/parquet/external_access_plan.rs
Original file line number Diff line number Diff line change
Expand Up @@ -346,7 +346,7 @@ impl TestFull {
let source = if let Some(predicate) = predicate {
let df_schema = DFSchema::try_from(schema.clone())?;
let predicate = ctx.create_physical_expr(predicate, &df_schema)?;
Arc::new(ParquetSource::default().with_predicate(schema.clone(), predicate))
Arc::new(ParquetSource::default().with_predicate(predicate))
} else {
Arc::new(ParquetSource::default())
};
Expand Down
2 changes: 1 addition & 1 deletion datafusion/core/tests/parquet/page_pruning.rs
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,7 @@ async fn get_parquet_exec(state: &SessionState, filter: Expr) -> DataSourceExec

let source = Arc::new(
ParquetSource::default()
.with_predicate(Arc::clone(&schema), predicate)
.with_predicate(predicate)
.with_enable_page_index(true),
);
let base_config = FileScanConfigBuilder::new(object_store_url, schema, source)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -99,7 +99,7 @@ impl FileSource for TestSource {
}

fn with_schema(&self, _schema: SchemaRef) -> Arc<dyn FileSource> {
todo!("should not be called")
Arc::new(self.clone()) as Arc<dyn FileSource>
}

fn with_projection(&self, _config: &FileScanConfig) -> Arc<dyn FileSource> {
Expand Down
29 changes: 2 additions & 27 deletions datafusion/datasource-parquet/src/file_format.rs
Original file line number Diff line number Diff line change
Expand Up @@ -29,9 +29,7 @@ use datafusion_datasource::file_compression_type::FileCompressionType;
use datafusion_datasource::file_sink_config::{FileSink, FileSinkConfig};
use datafusion_datasource::write::{create_writer, get_writer_schema, SharedBuffer};

use datafusion_datasource::file_format::{
FileFormat, FileFormatFactory, FilePushdownSupport,
};
use datafusion_datasource::file_format::{FileFormat, FileFormatFactory};
use datafusion_datasource::write::demux::DemuxedStreamReceiver;

use arrow::compute::sum;
Expand All @@ -52,15 +50,13 @@ use datafusion_datasource::sink::{DataSink, DataSinkExec};
use datafusion_execution::memory_pool::{MemoryConsumer, MemoryPool, MemoryReservation};
use datafusion_execution::{SendableRecordBatchStream, TaskContext};
use datafusion_expr::dml::InsertOp;
use datafusion_expr::Expr;
use datafusion_functions_aggregate::min_max::{MaxAccumulator, MinAccumulator};
use datafusion_physical_expr::PhysicalExpr;
use datafusion_physical_expr_common::sort_expr::LexRequirement;
use datafusion_physical_plan::Accumulator;
use datafusion_physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan};
use datafusion_session::Session;

use crate::can_expr_be_pushed_down_with_schemas;
use crate::source::{parse_coerce_int96_string, ParquetSource};
use async_trait::async_trait;
use bytes::Bytes;
Expand Down Expand Up @@ -431,7 +427,7 @@ impl FileFormat for ParquetFormat {
let mut source = ParquetSource::new(self.options.clone());

if let Some(predicate) = predicate {
source = source.with_predicate(Arc::clone(&conf.file_schema), predicate);
source = source.with_predicate(predicate);
}
if let Some(metadata_size_hint) = metadata_size_hint {
source = source.with_metadata_size_hint(metadata_size_hint)
Expand Down Expand Up @@ -459,27 +455,6 @@ impl FileFormat for ParquetFormat {
Ok(Arc::new(DataSinkExec::new(input, sink, order_requirements)) as _)
}

fn supports_filters_pushdown(
&self,
file_schema: &Schema,
table_schema: &Schema,
filters: &[&Expr],
) -> Result<FilePushdownSupport> {
if !self.options().global.pushdown_filters {
return Ok(FilePushdownSupport::NoSupport);
}

let all_supported = filters.iter().all(|filter| {
can_expr_be_pushed_down_with_schemas(filter, file_schema, table_schema)
});

Ok(if all_supported {
FilePushdownSupport::Supported
} else {
FilePushdownSupport::NotSupportedForFilter
})
}

fn file_source(&self) -> Arc<dyn FileSource> {
Arc::new(ParquetSource::default())
}
Expand Down
6 changes: 2 additions & 4 deletions datafusion/datasource-parquet/src/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -59,7 +59,6 @@ pub use metrics::ParquetFileMetrics;
pub use page_filter::PagePruningAccessPlanFilter;
pub use reader::{DefaultParquetFileReaderFactory, ParquetFileReaderFactory};
pub use row_filter::build_row_filter;
pub use row_filter::can_expr_be_pushed_down_with_schemas;
pub use row_group_filter::RowGroupAccessPlanFilter;
use source::ParquetSource;
pub use writer::plan_to_parquet;
Expand Down Expand Up @@ -223,8 +222,7 @@ impl ParquetExecBuilder {
} = self;
let mut parquet = ParquetSource::new(table_parquet_options);
if let Some(predicate) = predicate.clone() {
parquet = parquet
.with_predicate(Arc::clone(&file_scan_config.file_schema), predicate);
parquet = parquet.with_predicate(predicate);
}
if let Some(metadata_size_hint) = metadata_size_hint {
parquet = parquet.with_metadata_size_hint(metadata_size_hint)
Expand All @@ -244,7 +242,7 @@ impl ParquetExecBuilder {
inner: DataSourceExec::new(Arc::new(base_config.clone())),
base_config,
predicate,
pruning_predicate: parquet.pruning_predicate,
pruning_predicate: None, // for backwards compat since `ParquetExec` is only for backwards compat anyway
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Open to other suggestions (i.e. removing it). I felt like this minimizes breakage for folks still using ParquetExec, who are likely the same folks that want to do the least amount of work possible to upgrade.

schema_adapter_factory: parquet.schema_adapter_factory,
parquet_file_reader_factory: parquet.parquet_file_reader_factory,
table_parquet_options: parquet.table_parquet_options,
Expand Down
Loading