diff --git a/benchmarks/src/bin/tpch.rs b/benchmarks/src/bin/tpch.rs index b2f2bf1817877..71257d7b9647c 100644 --- a/benchmarks/src/bin/tpch.rs +++ b/benchmarks/src/bin/tpch.rs @@ -431,7 +431,8 @@ async fn get_table( } "parquet" => { let path = format!("{}/{}", path, table); - let format = ParquetFormat::default().with_enable_pruning(true); + let format = ParquetFormat::new(ctx.config.config_options()) + .with_enable_pruning(true); (Arc::new(format), path, DEFAULT_PARQUET_EXTENSION) } diff --git a/datafusion/core/src/config.rs b/datafusion/core/src/config.rs index 134e37390944c..b7a328983ea20 100644 --- a/datafusion/core/src/config.rs +++ b/datafusion/core/src/config.rs @@ -250,7 +250,8 @@ impl BuiltInConfigs { ConfigDefinition::new_bool( OPT_PARQUET_ENABLE_PRUNING, "If true, the parquet reader attempts to skip entire row groups based \ - on the predicate in the query.", + on the predicate in the query and the metadata (min/max values) stored in \ + the parquet file.", true, ), ConfigDefinition::new_bool( diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 5c97a34f74394..668aeea609ea0 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -618,8 +618,7 @@ mod tests { let (meta, _files) = store_parquet(vec![batch1, batch2]).await?; let ctx = SessionContext::new(); - let config_options = ctx.config_options(); - let format = ParquetFormat::new(config_options); + let format = ParquetFormat::new(ctx.config_options()); let schema = format.infer_schema(&store, &meta).await.unwrap(); let stats = @@ -767,11 +766,7 @@ mod tests { assert_eq!(store.request_count(), 2); let ctx = SessionContext::new(); - let config_options = ctx.config_options(); - config_options - .write() - .set_u64(OPT_PARQUET_METADATA_SIZE_HINT, 9); - let format = ParquetFormat::default(config_options); + let format = ParquetFormat::new(ctx.config_options()).with_metadata_size_hint(9); let schema = format.infer_schema(&store.upcast(), &meta).await.unwrap(); let stats = @@ -798,7 +793,8 @@ mod tests { // ensure the requests were coalesced into a single request assert_eq!(store.request_count(), 1); - let format = ParquetFormat::default().with_metadata_size_hint(size_hint); + let format = + ParquetFormat::new(ctx.config_options()).with_metadata_size_hint(size_hint); let schema = format.infer_schema(&store.upcast(), &meta).await.unwrap(); let stats = fetch_statistics( store.upcast().as_ref(), diff --git a/datafusion/core/src/physical_plan/file_format/parquet.rs b/datafusion/core/src/physical_plan/file_format/parquet.rs index 68b00a3ef08ce..ad3185003cba0 100644 --- a/datafusion/core/src/physical_plan/file_format/parquet.rs +++ b/datafusion/core/src/physical_plan/file_format/parquet.rs @@ -1164,6 +1164,7 @@ mod tests { use crate::config::ConfigOptions; use crate::datasource::file_format::parquet::test_util::store_parquet; use crate::datasource::file_format::test_util::scan_format; + use crate::datasource::file_format::FileFormat; use crate::datasource::listing::{FileRange, PartitionedFile}; use crate::datasource::object_store::ObjectStoreUrl; use crate::execution::options::CsvReadOptions; @@ -1660,7 +1661,7 @@ mod tests { async fn parquet_exec_with_projection() -> Result<()> { let testdata = crate::test_util::parquet_test_data(); let filename = "alltypes_plain.parquet"; - let format = ParquetFormat::default(); + let format = ParquetFormat::new(ConfigOptions::new().into_shareable()); let parquet_exec = scan_format(&format, &testdata, filename, Some(vec![0, 1, 2]), None) .await @@ -1742,7 +1743,7 @@ mod tests { let meta = local_unpartitioned_file(filename); let store = Arc::new(LocalFileSystem::new()) as _; - let file_schema = ParquetFormat::default() + let file_schema = ParquetFormat::new(session_ctx.config_options()) .infer_schema(&store, &[meta.clone()]) .await?; @@ -1789,7 +1790,7 @@ mod tests { let meta = local_unpartitioned_file(filename); - let schema = ParquetFormat::default() + let schema = ParquetFormat::new(session_ctx.config_options()) .infer_schema(&store, &[meta.clone()]) .await .unwrap(); @@ -2477,7 +2478,7 @@ mod tests { let meta = local_unpartitioned_file(filename); - let schema = ParquetFormat::default() + let schema = ParquetFormat::new(session_ctx.config_options()) .infer_schema(&store, &[meta.clone()]) .await .unwrap(); diff --git a/datafusion/proto/src/logical_plan.rs b/datafusion/proto/src/logical_plan.rs index d61bb2d65baea..74e04752a5b91 100644 --- a/datafusion/proto/src/logical_plan.rs +++ b/datafusion/proto/src/logical_plan.rs @@ -368,7 +368,8 @@ impl AsLogicalPlan for LogicalPlanNode { &FileFormatType::Parquet(protobuf::ParquetFormat { enable_pruning, }) => Arc::new( - ParquetFormat::default().with_enable_pruning(enable_pruning), + ParquetFormat::new(ctx.config_options()) + .with_enable_pruning(enable_pruning), ), FileFormatType::Csv(protobuf::CsvFormat { has_header,