Skip to content

Commit 5e1e693

Browse files
author
mertak-synnada
authored
Introduce unified DataSourceExec for provided datasources, remove ParquetExec, CsvExec, etc (#14224)
* unify ParquetExec, AvroExec, ArrowExec, NDJsonExec, MemoryExec into one DataSourceExec plan * fix license headers * fix compile errors on documents * separate non-parquet code * format code * fix typo * fix imports * fix clippy fix csv_json example * add comment to the example * fix cargo docs * change MemoryExec with MemorySourceConfig * merge fixes * change MemoryExec to DataSourceExec * fix merge conflicts * apply some syntactic sugars * fix imports and comment line * simplify some lines * rename source_config as file_source * format code * format code * make memory metrics default behavior * remove unnecessary cfg check * format code * remove ParquetExec strings * fix documents and imports * fix imports * add constraints and fix tests * delete redundant file * make metrics and statistics a part of File type specific configurations make cache a part of DataSourceExec * format code * fix tests * format code * split repartitioning into DataSourceExec and FileSourceConfig parts * move properties into DataSourceExec and split eq_properties and output_partitioning in DataSource trait * clone source with Arc * return file type as enum and do not downcast if not necessary create fmt_extra method * format code * re-add deprecated plans in order to support backward compatibility * reduce diff * fix doc * merge fixes * remove unnecessary files * rename config structs to source * remove empty files fix tests * removed FileSourceConfig projected_statistics must be solved! * fix base_config formatting * format code * fix repartition logic * fix merge conflicts * fix csv projection error * clippy fix * use new() on initialization * use DataSourceExec on deprecated file operators as well * move ParquetSource into source.rs fix doc errors * use ParquetSource only if parquet feature is enabled * fix slt tests * add with_fetch API to MemorySourceConfig and re-add deprecated MemoryExec * fix merge conflicts fix memory source fetch error * format code * change FileType enum into a dyn Trait so that it can be extensible * remove metadata_size_hint from required ParquetSource parameters * remove FileType trait and split with_predicate logic for ParquetSource * remove predicate from initialization of ParquetSource * remove unnecessary imports * deprecate ParquetExecBuilder and add doc hints * fix slt * fix clippy * fix fmt * return reference of the Arc in source() * re-add deprecated exec files * fix doc error
1 parent 8ebed67 commit 5e1e693

File tree

170 files changed

+4868
-3460
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

170 files changed

+4868
-3460
lines changed

datafusion-cli/src/functions.rs

Lines changed: 11 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -16,29 +16,31 @@
1616
// under the License.
1717

1818
//! Functions that are query-able and searchable via the `\h` command
19+
20+
use std::fmt;
21+
use std::fs::File;
22+
use std::str::FromStr;
23+
use std::sync::Arc;
24+
1925
use arrow::array::{Int64Array, StringArray};
2026
use arrow::datatypes::{DataType, Field, Schema, SchemaRef};
2127
use arrow::record_batch::RecordBatch;
2228
use arrow::util::pretty::pretty_format_batches;
23-
use async_trait::async_trait;
24-
2529
use datafusion::catalog::{Session, TableFunctionImpl};
2630
use datafusion::common::{plan_err, Column};
2731
use datafusion::datasource::TableProvider;
2832
use datafusion::error::Result;
2933
use datafusion::logical_expr::Expr;
30-
use datafusion::physical_plan::memory::MemoryExec;
34+
use datafusion::physical_plan::memory::MemorySourceConfig;
3135
use datafusion::physical_plan::ExecutionPlan;
3236
use datafusion::scalar::ScalarValue;
37+
38+
use async_trait::async_trait;
3339
use parquet::basic::ConvertedType;
3440
use parquet::data_type::{ByteArray, FixedLenByteArray};
3541
use parquet::file::reader::FileReader;
3642
use parquet::file::serialized_reader::SerializedFileReader;
3743
use parquet::file::statistics::Statistics;
38-
use std::fmt;
39-
use std::fs::File;
40-
use std::str::FromStr;
41-
use std::sync::Arc;
4244

4345
#[derive(Debug)]
4446
pub enum Function {
@@ -240,11 +242,11 @@ impl TableProvider for ParquetMetadataTable {
240242
_filters: &[Expr],
241243
_limit: Option<usize>,
242244
) -> Result<Arc<dyn ExecutionPlan>> {
243-
Ok(Arc::new(MemoryExec::try_new(
245+
Ok(MemorySourceConfig::try_new_exec(
244246
&[vec![self.batch.clone()]],
245247
TableProvider::schema(self),
246248
projection.cloned(),
247-
)?))
249+
)?)
248250
}
249251
}
250252

datafusion-examples/examples/advanced_parquet_index.rs

Lines changed: 47 additions & 45 deletions
Original file line numberDiff line numberDiff line change
@@ -25,18 +25,14 @@ use std::sync::Arc;
2525

2626
use arrow::array::{ArrayRef, Int32Array, RecordBatch, StringArray};
2727
use arrow::datatypes::SchemaRef;
28-
use async_trait::async_trait;
29-
use bytes::Bytes;
3028
use datafusion::catalog::Session;
3129
use datafusion::common::{
3230
internal_datafusion_err, DFSchema, DataFusionError, Result, ScalarValue,
3331
};
3432
use datafusion::datasource::listing::PartitionedFile;
35-
use datafusion::datasource::physical_plan::parquet::{
36-
ParquetAccessPlan, ParquetExecBuilder,
37-
};
33+
use datafusion::datasource::physical_plan::parquet::ParquetAccessPlan;
3834
use datafusion::datasource::physical_plan::{
39-
parquet::ParquetFileReaderFactory, FileMeta, FileScanConfig,
35+
parquet::ParquetFileReaderFactory, FileMeta, FileScanConfig, ParquetSource,
4036
};
4137
use datafusion::datasource::TableProvider;
4238
use datafusion::execution::object_store::ObjectStoreUrl;
@@ -56,6 +52,9 @@ use datafusion::physical_optimizer::pruning::PruningPredicate;
5652
use datafusion::physical_plan::metrics::ExecutionPlanMetricsSet;
5753
use datafusion::physical_plan::ExecutionPlan;
5854
use datafusion::prelude::*;
55+
56+
use async_trait::async_trait;
57+
use bytes::Bytes;
5958
use futures::future::BoxFuture;
6059
use futures::FutureExt;
6160
use object_store::ObjectStore;
@@ -83,8 +82,8 @@ use url::Url;
8382
/// Specifically, this example illustrates how to:
8483
/// 1. Use [`ParquetFileReaderFactory`] to avoid re-reading parquet metadata on each query
8584
/// 2. Use [`PruningPredicate`] for predicate analysis
86-
/// 3. Pass a row group selection to [`ParquetExec`]
87-
/// 4. Pass a row selection (within a row group) to [`ParquetExec`]
85+
/// 3. Pass a row group selection to [`ParquetSource`]
86+
/// 4. Pass a row selection (within a row group) to [`ParquetSource`]
8887
///
8988
/// Note this is a *VERY* low level example for people who want to build their
9089
/// own custom indexes (e.g. for low latency queries). Most users should use
@@ -94,38 +93,38 @@ use url::Url;
9493
///
9594
/// # Diagram
9695
///
97-
/// This diagram shows how the `ParquetExec` is configured to do only a single
96+
/// This diagram shows how the `DataSourceExec` with `ParquetSource` is configured to do only a single
9897
/// (range) read from a parquet file, for the data that is needed. It does
9998
/// not read the file footer or any of the row groups that are not needed.
10099
///
101100
/// ```text
102101
/// ┌───────────────────────┐ The TableProvider configures the
103-
/// │ ┌───────────────────┐ │ ParquetExec:
102+
/// │ ┌───────────────────┐ │ DataSourceExec:
104103
/// │ │ │ │
105104
/// │ └───────────────────┘ │
106105
/// │ ┌───────────────────┐ │
107106
/// Row │ │ │ │ 1. To read only specific Row
108-
/// Groups │ └───────────────────┘ │ Groups (the ParquetExec tries
107+
/// Groups │ └───────────────────┘ │ Groups (the DataSourceExec tries
109108
/// │ ┌───────────────────┐ │ to reduce this further based
110109
/// │ │ │ │ on metadata)
111-
/// │ └───────────────────┘ │ ┌────────────────────┐
112-
/// │ ┌───────────────────┐ │ │ │
113-
/// │ │ │◀┼ ─ ─ ┐ │ ParquetExec
114-
/// │ └───────────────────┘ │ │ (Parquet Reader) │
115-
/// │ ... │ └ ─ ─ ─ ─│ │
116-
/// │ ┌───────────────────┐ │ │ ╔═══════════════╗ │
117-
/// │ │ │ │ │ ║ParquetMetadata║ │
118-
/// │ └───────────────────┘ │ │ ╚═══════════════╝ │
119-
/// │ ╔═══════════════════╗ │ └────────────────────┘
110+
/// │ └───────────────────┘ │ ┌──────────────────────
111+
/// │ ┌───────────────────┐ │ │
112+
/// │ │ │◀┼ ─ ─ ┐ │ DataSourceExec
113+
/// │ └───────────────────┘ │ │ (Parquet Reader)
114+
/// │ ... │ └ ─ ─ ─ ─│
115+
/// │ ┌───────────────────┐ │ │ ╔═══════════════╗
116+
/// │ │ │ │ │ ║ParquetMetadata║
117+
/// │ └───────────────────┘ │ │ ╚═══════════════╝
118+
/// │ ╔═══════════════════╗ │ └──────────────────────
120119
/// │ ║ Thrift metadata ║ │
121120
/// │ ╚═══════════════════╝ │ 1. With cached ParquetMetadata, so
122-
/// └───────────────────────┘ the ParquetExec does not re-read /
121+
/// └───────────────────────┘ the ParquetSource does not re-read /
123122
/// Parquet File decode the thrift footer
124123
///
125124
/// ```
126125
///
127126
/// Within a Row Group, Column Chunks store data in DataPages. This example also
128-
/// shows how to configure the ParquetExec to read a `RowSelection` (row ranges)
127+
/// shows how to configure the ParquetSource to read a `RowSelection` (row ranges)
129128
/// which will skip unneeded data pages. This requires that the Parquet file has
130129
/// a [Page Index].
131130
///
@@ -135,15 +134,15 @@ use url::Url;
135134
/// │ │ Data Page is not fetched or decoded.
136135
/// │ ┌───────────────────┐ │ Note this requires a PageIndex
137136
/// │ │ ┌──────────┐ │ │
138-
/// Row │ │ │DataPage 0│ │ │ ┌────────────────────┐
139-
/// Groups │ │ └──────────┘ │ │ │ │
140-
/// │ │ ┌──────────┐ │ │ │ ParquetExec
141-
/// │ │ ... │DataPage 1│ ◀┼ ┼ ─ ─ ─ │ (Parquet Reader) │
142-
/// │ │ └──────────┘ │ │ └ ─ ─ ─ ─ ─│ │
143-
/// │ │ ┌──────────┐ │ │ │ ╔═══════════════╗ │
144-
/// │ │ │DataPage 2│ │ │ If only rows │ ║ParquetMetadata║ │
145-
/// │ │ └──────────┘ │ │ from DataPage 1 │ ╚═══════════════╝ │
146-
/// │ └───────────────────┘ │ are selected, └────────────────────┘
137+
/// Row │ │ │DataPage 0│ │ │ ┌──────────────────────
138+
/// Groups │ │ └──────────┘ │ │ │
139+
/// │ │ ┌──────────┐ │ │ │ DataSourceExec
140+
/// │ │ ... │DataPage 1│ ◀┼ ┼ ─ ─ ─ │ (Parquet Reader)
141+
/// │ │ └──────────┘ │ │ └ ─ ─ ─ ─ ─│
142+
/// │ │ ┌──────────┐ │ │ │ ╔═══════════════╗
143+
/// │ │ │DataPage 2│ │ │ If only rows │ ║ParquetMetadata║
144+
/// │ │ └──────────┘ │ │ from DataPage 1 │ ╚═══════════════╝
145+
/// │ └───────────────────┘ │ are selected, └──────────────────────
147146
/// │ │ only DataPage 1
148147
/// │ ... │ is fetched and
149148
/// │ │ decoded
@@ -211,7 +210,7 @@ async fn main() -> Result<()> {
211210
// pages that must be decoded
212211
//
213212
// Note: in order to prune pages, the Page Index must be loaded and the
214-
// ParquetExec will load it on demand if not present. To avoid a second IO
213+
// DataSourceExec will load it on demand if not present. To avoid a second IO
215214
// during query, this example loaded the Page Index preemptively by setting
216215
// `ArrowReader::with_page_index` in `IndexedFile::try_new`
217216
provider.set_use_row_selection(true);
@@ -478,31 +477,34 @@ impl TableProvider for IndexTableProvider {
478477

479478
let partitioned_file = indexed_file
480479
.partitioned_file()
481-
// provide the starting access plan to the ParquetExec by
480+
// provide the starting access plan to the DataSourceExec by
482481
// storing it as "extensions" on PartitionedFile
483482
.with_extensions(Arc::new(access_plan) as _);
484483

485484
// Prepare for scanning
486485
let schema = self.schema();
487486
let object_store_url = ObjectStoreUrl::parse("file://")?;
488-
let file_scan_config = FileScanConfig::new(object_store_url, schema)
489-
.with_limit(limit)
490-
.with_projection(projection.cloned())
491-
.with_file(partitioned_file);
492487

493488
// Configure a factory interface to avoid re-reading the metadata for each file
494489
let reader_factory =
495490
CachedParquetFileReaderFactory::new(Arc::clone(&self.object_store))
496491
.with_file(indexed_file);
497492

498-
// Finally, put it all together into a ParquetExec
499-
Ok(ParquetExecBuilder::new(file_scan_config)
500-
// provide the predicate so the ParquetExec can try and prune
501-
// row groups internally
502-
.with_predicate(predicate)
503-
// provide the factory to create parquet reader without re-reading metadata
504-
.with_parquet_file_reader_factory(Arc::new(reader_factory))
505-
.build_arc())
493+
let file_source = Arc::new(
494+
ParquetSource::default()
495+
// provide the predicate so the DataSourceExec can try and prune
496+
// row groups internally
497+
.with_predicate(Arc::clone(&schema), predicate)
498+
// provide the factory to create parquet reader without re-reading metadata
499+
.with_parquet_file_reader_factory(Arc::new(reader_factory)),
500+
);
501+
let file_scan_config = FileScanConfig::new(object_store_url, schema, file_source)
502+
.with_limit(limit)
503+
.with_projection(projection.cloned())
504+
.with_file(partitioned_file);
505+
506+
// Finally, put it all together into a DataSourceExec
507+
Ok(file_scan_config.new_exec())
506508
}
507509

508510
/// Tell DataFusion to push filters down to the scan method

datafusion-examples/examples/csv_json_opener.rs

Lines changed: 35 additions & 26 deletions
Original file line numberDiff line numberDiff line change
@@ -18,18 +18,21 @@
1818
use std::sync::Arc;
1919

2020
use arrow::datatypes::{DataType, Field, Schema};
21+
use datafusion::datasource::physical_plan::JsonSource;
2122
use datafusion::{
2223
assert_batches_eq,
2324
datasource::{
25+
data_source::FileSource,
2426
file_format::file_compression_type::FileCompressionType,
2527
listing::PartitionedFile,
2628
object_store::ObjectStoreUrl,
27-
physical_plan::{CsvConfig, CsvOpener, FileScanConfig, FileStream, JsonOpener},
29+
physical_plan::{CsvSource, FileScanConfig, FileStream, JsonOpener},
2830
},
2931
error::Result,
3032
physical_plan::metrics::ExecutionPlanMetricsSet,
3133
test_util::aggr_test_schema,
3234
};
35+
3336
use futures::StreamExt;
3437
use object_store::{local::LocalFileSystem, memory::InMemory, ObjectStore};
3538

@@ -48,29 +51,27 @@ async fn csv_opener() -> Result<()> {
4851
let object_store = Arc::new(LocalFileSystem::new());
4952
let schema = aggr_test_schema();
5053

51-
let config = CsvConfig::new(
52-
8192,
53-
schema.clone(),
54-
Some(vec![12, 0]),
55-
true,
56-
b',',
57-
b'"',
58-
None,
59-
object_store,
60-
Some(b'#'),
61-
);
62-
63-
let opener = CsvOpener::new(Arc::new(config), FileCompressionType::UNCOMPRESSED);
64-
6554
let testdata = datafusion::test_util::arrow_test_data();
6655
let path = format!("{testdata}/csv/aggregate_test_100.csv");
6756

6857
let path = std::path::Path::new(&path).canonicalize()?;
6958

70-
let scan_config = FileScanConfig::new(ObjectStoreUrl::local_filesystem(), schema)
71-
.with_projection(Some(vec![12, 0]))
72-
.with_limit(Some(5))
73-
.with_file(PartitionedFile::new(path.display().to_string(), 10));
59+
let scan_config = FileScanConfig::new(
60+
ObjectStoreUrl::local_filesystem(),
61+
Arc::clone(&schema),
62+
Arc::new(CsvSource::default()),
63+
)
64+
.with_projection(Some(vec![12, 0]))
65+
.with_limit(Some(5))
66+
.with_file(PartitionedFile::new(path.display().to_string(), 10));
67+
68+
let config = CsvSource::new(true, b',', b'"')
69+
.with_comment(Some(b'#'))
70+
.with_schema(schema)
71+
.with_batch_size(8192)
72+
.with_projection(&scan_config);
73+
74+
let opener = config.create_file_opener(Ok(object_store), &scan_config, 0)?;
7475

7576
let mut result = vec![];
7677
let mut stream =
@@ -120,13 +121,21 @@ async fn json_opener() -> Result<()> {
120121
Arc::new(object_store),
121122
);
122123

123-
let scan_config = FileScanConfig::new(ObjectStoreUrl::local_filesystem(), schema)
124-
.with_projection(Some(vec![1, 0]))
125-
.with_limit(Some(5))
126-
.with_file(PartitionedFile::new(path.to_string(), 10));
127-
128-
let mut stream =
129-
FileStream::new(&scan_config, 0, opener, &ExecutionPlanMetricsSet::new())?;
124+
let scan_config = FileScanConfig::new(
125+
ObjectStoreUrl::local_filesystem(),
126+
schema,
127+
Arc::new(JsonSource::default()),
128+
)
129+
.with_projection(Some(vec![1, 0]))
130+
.with_limit(Some(5))
131+
.with_file(PartitionedFile::new(path.to_string(), 10));
132+
133+
let mut stream = FileStream::new(
134+
&scan_config,
135+
0,
136+
Arc::new(opener),
137+
&ExecutionPlanMetricsSet::new(),
138+
)?;
130139
let mut result = vec![];
131140
while let Some(batch) = stream.next().await.transpose()? {
132141
result.push(batch);

datafusion-examples/examples/custom_file_format.rs

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@ use arrow::{
2222
datatypes::{DataType, Field, Schema, SchemaRef, UInt64Type},
2323
};
2424
use datafusion::common::{GetExt, Statistics};
25+
use datafusion::datasource::data_source::FileSource;
2526
use datafusion::execution::session_state::SessionStateBuilder;
2627
use datafusion::physical_expr::LexRequirement;
2728
use datafusion::physical_expr::PhysicalExpr;
@@ -126,6 +127,10 @@ impl FileFormat for TSVFileFormat {
126127
.create_writer_physical_plan(input, state, conf, order_requirements)
127128
.await
128129
}
130+
131+
fn file_source(&self) -> Arc<dyn FileSource> {
132+
self.csv_file_format.file_source()
133+
}
129134
}
130135

131136
#[derive(Default, Debug)]

datafusion-examples/examples/parquet_exec_visitor.rs

Lines changed: 19 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -19,10 +19,11 @@ use std::sync::Arc;
1919

2020
use datafusion::datasource::file_format::parquet::ParquetFormat;
2121
use datafusion::datasource::listing::{ListingOptions, PartitionedFile};
22-
use datafusion::datasource::physical_plan::ParquetExec;
22+
use datafusion::datasource::physical_plan::{FileScanConfig, ParquetSource};
2323
use datafusion::error::DataFusionError;
2424
use datafusion::execution::context::SessionContext;
2525
use datafusion::physical_plan::metrics::MetricValue;
26+
use datafusion::physical_plan::source::DataSourceExec;
2627
use datafusion::physical_plan::{
2728
execute_stream, visit_execution_plan, ExecutionPlan, ExecutionPlanVisitor,
2829
};
@@ -96,15 +97,24 @@ impl ExecutionPlanVisitor for ParquetExecVisitor {
9697
/// or `post_visit` (visit each node after its children/inputs)
9798
fn pre_visit(&mut self, plan: &dyn ExecutionPlan) -> Result<bool, Self::Error> {
9899
// If needed match on a specific `ExecutionPlan` node type
99-
let maybe_parquet_exec = plan.as_any().downcast_ref::<ParquetExec>();
100-
if let Some(parquet_exec) = maybe_parquet_exec {
101-
self.file_groups = Some(parquet_exec.base_config().file_groups.clone());
100+
if let Some(data_source) = plan.as_any().downcast_ref::<DataSourceExec>() {
101+
let source = data_source.source();
102+
if let Some(file_config) = source.as_any().downcast_ref::<FileScanConfig>() {
103+
if file_config
104+
.file_source()
105+
.as_any()
106+
.downcast_ref::<ParquetSource>()
107+
.is_some()
108+
{
109+
self.file_groups = Some(file_config.file_groups.clone());
102110

103-
let metrics = match parquet_exec.metrics() {
104-
None => return Ok(true),
105-
Some(metrics) => metrics,
106-
};
107-
self.bytes_scanned = metrics.sum_by_name("bytes_scanned");
111+
let metrics = match data_source.metrics() {
112+
None => return Ok(true),
113+
Some(metrics) => metrics,
114+
};
115+
self.bytes_scanned = metrics.sum_by_name("bytes_scanned");
116+
}
117+
}
108118
}
109119
Ok(true)
110120
}

0 commit comments

Comments
 (0)