-
Notifications
You must be signed in to change notification settings - Fork 1.6k
Redesign ownership model between FileScanConfig
and FileSource
s
#17242
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
base: main
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -55,6 +55,7 @@ use arrow::array::{ArrayRef, Int32Array, RecordBatch, StringArray}; | |
use arrow::datatypes::SchemaRef; | ||
use async_trait::async_trait; | ||
use bytes::Bytes; | ||
use datafusion::config::TableParquetOptions; | ||
use datafusion::datasource::memory::DataSourceExec; | ||
use futures::future::BoxFuture; | ||
use futures::FutureExt; | ||
|
@@ -491,23 +492,22 @@ impl TableProvider for IndexTableProvider { | |
CachedParquetFileReaderFactory::new(Arc::clone(&self.object_store)) | ||
.with_file(indexed_file); | ||
|
||
let file_source = Arc::new( | ||
ParquetSource::default() | ||
let file_scan_config = FileScanConfigBuilder::new(object_store_url, schema) | ||
.with_limit(limit) | ||
.with_projection(projection.cloned()) | ||
.with_file(partitioned_file) | ||
.build(); | ||
|
||
let file_source = | ||
ParquetSource::new(TableParquetOptions::default(), file_scan_config.clone()) | ||
// provide the predicate so the DataSourceExec can try and prune | ||
// row groups internally | ||
.with_predicate(predicate) | ||
// provide the factory to create parquet reader without re-reading metadata | ||
.with_parquet_file_reader_factory(Arc::new(reader_factory)), | ||
); | ||
let file_scan_config = | ||
FileScanConfigBuilder::new(object_store_url, schema, file_source) | ||
.with_limit(limit) | ||
.with_projection(projection.cloned()) | ||
.with_file(partitioned_file) | ||
.build(); | ||
.with_parquet_file_reader_factory(Arc::new(reader_factory)); | ||
|
||
// Finally, put it all together into a DataSourceExec | ||
Ok(DataSourceExec::from_data_source(file_scan_config)) | ||
Comment on lines
-494
to
-510
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. You'll see this pattern a lot in this PR. The old flow goes something like:
Inside this flow, there's a circular dependency (call file source from config, create file opener from file source but also pass in the config). The new flow goes something like:
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The biggest point of the refactor |
||
Ok(DataSourceExec::from_data_source(file_source)) | ||
} | ||
|
||
/// Tell DataFusion to push filters down to the scan method | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -24,7 +24,7 @@ use datafusion::{ | |
file_format::file_compression_type::FileCompressionType, | ||
listing::PartitionedFile, | ||
object_store::ObjectStoreUrl, | ||
physical_plan::{CsvSource, FileSource, FileStream, JsonOpener, JsonSource}, | ||
physical_plan::{CsvSource, FileSource, FileStream, JsonOpener}, | ||
}, | ||
error::Result, | ||
physical_plan::metrics::ExecutionPlanMetricsSet, | ||
|
@@ -58,20 +58,17 @@ async fn csv_opener() -> Result<()> { | |
let scan_config = FileScanConfigBuilder::new( | ||
ObjectStoreUrl::local_filesystem(), | ||
Arc::clone(&schema), | ||
Arc::new(CsvSource::default()), | ||
) | ||
.with_projection(Some(vec![12, 0])) | ||
.with_batch_size(Some(8192)) | ||
.with_limit(Some(5)) | ||
.with_file(PartitionedFile::new(path.display().to_string(), 10)) | ||
.build(); | ||
|
||
let config = CsvSource::new(true, b',', b'"') | ||
.with_comment(Some(b'#')) | ||
.with_schema(schema) | ||
.with_batch_size(8192) | ||
.with_projection(&scan_config); | ||
Comment on lines
-70
to
-72
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It makes sense that the schema, batch size and projection are not properties inherent to CSVs and thus should be part of FileScanConfig. In fact they are currently duplicated! |
||
let source = | ||
CsvSource::new(true, b',', b'"', scan_config.clone()).with_comment(Some(b'#')); | ||
|
||
let opener = config.create_file_opener(object_store, &scan_config, 0); | ||
let opener = source.create_file_opener(object_store, 0); | ||
|
||
let mut result = vec![]; | ||
let mut stream = | ||
|
@@ -121,15 +118,12 @@ async fn json_opener() -> Result<()> { | |
Arc::new(object_store), | ||
); | ||
|
||
let scan_config = FileScanConfigBuilder::new( | ||
ObjectStoreUrl::local_filesystem(), | ||
schema, | ||
Arc::new(JsonSource::default()), | ||
) | ||
.with_projection(Some(vec![1, 0])) | ||
.with_limit(Some(5)) | ||
.with_file(PartitionedFile::new(path.to_string(), 10)) | ||
.build(); | ||
let scan_config = | ||
FileScanConfigBuilder::new(ObjectStoreUrl::local_filesystem(), schema) | ||
.with_projection(Some(vec![1, 0])) | ||
.with_limit(Some(5)) | ||
.with_file(PartitionedFile::new(path.to_string(), 10)) | ||
.build(); | ||
|
||
let mut stream = FileStream::new( | ||
&scan_config, | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -19,7 +19,7 @@ use std::sync::Arc; | |
|
||
use datafusion::datasource::file_format::parquet::ParquetFormat; | ||
use datafusion::datasource::listing::ListingOptions; | ||
use datafusion::datasource::physical_plan::{FileGroup, ParquetSource}; | ||
use datafusion::datasource::physical_plan::{FileGroup, FileSource, ParquetSource}; | ||
use datafusion::datasource::source::DataSourceExec; | ||
use datafusion::error::DataFusionError; | ||
use datafusion::execution::context::SessionContext; | ||
|
@@ -98,9 +98,11 @@ impl ExecutionPlanVisitor for ParquetExecVisitor { | |
fn pre_visit(&mut self, plan: &dyn ExecutionPlan) -> Result<bool, Self::Error> { | ||
// If needed match on a specific `ExecutionPlan` node type | ||
if let Some(data_source_exec) = plan.as_any().downcast_ref::<DataSourceExec>() { | ||
if let Some((file_config, _)) = | ||
if let Some(parquet_source) = | ||
data_source_exec.downcast_to_file_source::<ParquetSource>() | ||
{ | ||
let file_config = parquet_source.config(); | ||
|
||
self.file_groups = Some(file_config.file_groups.clone()); | ||
Comment on lines
+104
to
106
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This seems like a valid usage. Only question I have is if we should make There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Hm, interesting point. With 5d4e834, I'm starting to see a stratified API for The statistics getters were the only blatant cases where it was better served to have those functions live on Will think more about this |
||
|
||
let metrics = match data_source_exec.metrics() { | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is just moving code around to initialize the FileScanConfig before the ParquetSource ✅