From 62363a1298b73284310f60f6a1f441f385dec1f9 Mon Sep 17 00:00:00 2001 From: Jonathan Date: Thu, 7 Aug 2025 03:21:23 -0400 Subject: [PATCH 1/7] fix: Remove `datafusion.execution.parquet.cache_metadata` config --- datafusion/common/src/config.rs | 6 ----- .../common/src/file_options/parquet_writer.rs | 3 --- .../src/datasource/file_format/options.rs | 15 ------------ .../datasource-parquet/src/file_format.rs | 23 ++++++++----------- .../proto/datafusion_common.proto | 1 - datafusion/proto-common/src/from_proto/mod.rs | 1 - .../proto-common/src/generated/pbjson.rs | 16 ------------- .../proto-common/src/generated/prost.rs | 3 --- datafusion/proto-common/src/to_proto/mod.rs | 1 - .../src/generated/datafusion_proto_common.rs | 3 --- .../proto/src/logical_plan/file_formats.rs | 2 -- .../test_files/information_schema.slt | 2 -- .../sqllogictest/test_files/parquet.slt | 6 ----- docs/source/user-guide/configs.md | 3 +-- 14 files changed, 11 insertions(+), 74 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index be36f37d93b4..be2a734d37fc 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -559,12 +559,6 @@ config_namespace! { /// (reading) Use any available bloom filters when reading parquet files pub bloom_filter_on_read: bool, default = true - /// (reading) Whether or not to enable the caching of embedded metadata of Parquet files - /// (footer and page metadata). Enabling it can offer substantial performance improvements - /// for repeated queries over large files. By default, the cache is automatically - /// invalidated when the underlying file is modified. - pub cache_metadata: bool, default = false - // The following options affect writing to parquet files // and map to parquet::file::properties::WriterProperties diff --git a/datafusion/common/src/file_options/parquet_writer.rs b/datafusion/common/src/file_options/parquet_writer.rs index f7ae9b9d923d..91683ccb1b37 100644 --- a/datafusion/common/src/file_options/parquet_writer.rs +++ b/datafusion/common/src/file_options/parquet_writer.rs @@ -234,7 +234,6 @@ impl ParquetOptions { binary_as_string: _, // not used for writer props coerce_int96: _, // not used for writer props skip_arrow_metadata: _, - cache_metadata: _, } = self; let mut builder = WriterProperties::builder() @@ -502,7 +501,6 @@ mod tests { binary_as_string: defaults.binary_as_string, skip_arrow_metadata: defaults.skip_arrow_metadata, coerce_int96: None, - cache_metadata: defaults.cache_metadata, } } @@ -613,7 +611,6 @@ mod tests { binary_as_string: global_options_defaults.binary_as_string, skip_arrow_metadata: global_options_defaults.skip_arrow_metadata, coerce_int96: None, - cache_metadata: global_options_defaults.cache_metadata, }, column_specific_options, key_value_metadata, diff --git a/datafusion/core/src/datasource/file_format/options.rs b/datafusion/core/src/datasource/file_format/options.rs index 459e92a7a976..02b792823a82 100644 --- a/datafusion/core/src/datasource/file_format/options.rs +++ b/datafusion/core/src/datasource/file_format/options.rs @@ -254,11 +254,6 @@ pub struct ParquetReadOptions<'a> { pub file_sort_order: Vec>, /// Properties for decryption of Parquet files that use modular encryption pub file_decryption_properties: Option, - /// Whether or not to enable the caching of embedded metadata of this Parquet file (footer and - /// page metadata). Enabling it can offer substantial performance improvements for repeated - /// queries over large files. By default, the cache is automatically invalidated when the - /// underlying file is modified. - pub cache_metadata: Option, } impl Default for ParquetReadOptions<'_> { @@ -271,7 +266,6 @@ impl Default for ParquetReadOptions<'_> { schema: None, file_sort_order: vec![], file_decryption_properties: None, - cache_metadata: None, } } } @@ -331,12 +325,6 @@ impl<'a> ParquetReadOptions<'a> { self.file_decryption_properties = Some(file_decryption_properties); self } - - /// Specify whether to enable or not metadata caching - pub fn cache_metadata(mut self, cache_metadata: bool) -> Self { - self.cache_metadata = Some(cache_metadata); - self - } } /// Options that control the reading of ARROW files. @@ -602,9 +590,6 @@ impl ReadOptions<'_> for ParquetReadOptions<'_> { if let Some(file_decryption_properties) = &self.file_decryption_properties { options.crypto.file_decryption = Some(file_decryption_properties.clone()); } - if let Some(cache_metadata) = self.cache_metadata { - options.global.cache_metadata = cache_metadata; - } let mut file_format = ParquetFormat::new().with_options(options); if let Some(parquet_pruning) = self.parquet_pruning { diff --git a/datafusion/datasource-parquet/src/file_format.rs b/datafusion/datasource-parquet/src/file_format.rs index 7210cc09a0b3..582a0cd710fc 100644 --- a/datafusion/datasource-parquet/src/file_format.rs +++ b/datafusion/datasource-parquet/src/file_format.rs @@ -447,19 +447,16 @@ impl FileFormat for ParquetFormat { let mut source = ParquetSource::new(self.options.clone()); - // Use the CachedParquetFileReaderFactory when metadata caching is enabled - if self.options.global.cache_metadata { - if let Some(metadata_cache) = - state.runtime_env().cache_manager.get_file_metadata_cache() - { - let store = state - .runtime_env() - .object_store(conf.object_store_url.clone())?; - let cached_parquet_read_factory = - Arc::new(CachedParquetFileReaderFactory::new(store, metadata_cache)); - source = - source.with_parquet_file_reader_factory(cached_parquet_read_factory); - } + // Use the CachedParquetFileReaderFactory + if let Some(metadata_cache) = + state.runtime_env().cache_manager.get_file_metadata_cache() + { + let store = state + .runtime_env() + .object_store(conf.object_store_url.clone())?; + let cached_parquet_read_factory = + Arc::new(CachedParquetFileReaderFactory::new(store, metadata_cache)); + source = source.with_parquet_file_reader_factory(cached_parquet_read_factory); } if let Some(metadata_size_hint) = metadata_size_hint { diff --git a/datafusion/proto-common/proto/datafusion_common.proto b/datafusion/proto-common/proto/datafusion_common.proto index 5fb537160bac..f5c79cf3d9a4 100644 --- a/datafusion/proto-common/proto/datafusion_common.proto +++ b/datafusion/proto-common/proto/datafusion_common.proto @@ -530,7 +530,6 @@ message ParquetOptions { bool schema_force_view_types = 28; // default = false bool binary_as_string = 29; // default = false bool skip_arrow_metadata = 30; // default = false - bool cache_metadata = 33; // default = false oneof metadata_size_hint_opt { uint64 metadata_size_hint = 4; diff --git a/datafusion/proto-common/src/from_proto/mod.rs b/datafusion/proto-common/src/from_proto/mod.rs index 94024abfed44..c5242d0176e6 100644 --- a/datafusion/proto-common/src/from_proto/mod.rs +++ b/datafusion/proto-common/src/from_proto/mod.rs @@ -999,7 +999,6 @@ impl TryFrom<&protobuf::ParquetOptions> for ParquetOptions { protobuf::parquet_options::CoerceInt96Opt::CoerceInt96(v) => Some(v), }).unwrap_or(None), skip_arrow_metadata: value.skip_arrow_metadata, - cache_metadata: value.cache_metadata, }) } } diff --git a/datafusion/proto-common/src/generated/pbjson.rs b/datafusion/proto-common/src/generated/pbjson.rs index c92869655fed..61001ae3169e 100644 --- a/datafusion/proto-common/src/generated/pbjson.rs +++ b/datafusion/proto-common/src/generated/pbjson.rs @@ -5568,9 +5568,6 @@ impl serde::Serialize for ParquetOptions { if self.skip_arrow_metadata { len += 1; } - if self.cache_metadata { - len += 1; - } if self.dictionary_page_size_limit != 0 { len += 1; } @@ -5670,9 +5667,6 @@ impl serde::Serialize for ParquetOptions { if self.skip_arrow_metadata { struct_ser.serialize_field("skipArrowMetadata", &self.skip_arrow_metadata)?; } - if self.cache_metadata { - struct_ser.serialize_field("cacheMetadata", &self.cache_metadata)?; - } if self.dictionary_page_size_limit != 0 { #[allow(clippy::needless_borrow)] #[allow(clippy::needless_borrows_for_generic_args)] @@ -5810,7 +5804,6 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { "binaryAsString", "skip_arrow_metadata", "skipArrowMetadata", - "cache_metadata", "cacheMetadata", "dictionary_page_size_limit", "dictionaryPageSizeLimit", @@ -5910,7 +5903,6 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { "schemaForceViewTypes" | "schema_force_view_types" => Ok(GeneratedField::SchemaForceViewTypes), "binaryAsString" | "binary_as_string" => Ok(GeneratedField::BinaryAsString), "skipArrowMetadata" | "skip_arrow_metadata" => Ok(GeneratedField::SkipArrowMetadata), - "cacheMetadata" | "cache_metadata" => Ok(GeneratedField::CacheMetadata), "dictionaryPageSizeLimit" | "dictionary_page_size_limit" => Ok(GeneratedField::DictionaryPageSizeLimit), "dataPageRowCountLimit" | "data_page_row_count_limit" => Ok(GeneratedField::DataPageRowCountLimit), "maxRowGroupSize" | "max_row_group_size" => Ok(GeneratedField::MaxRowGroupSize), @@ -5960,7 +5952,6 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { let mut schema_force_view_types__ = None; let mut binary_as_string__ = None; let mut skip_arrow_metadata__ = None; - let mut cache_metadata__ = None; let mut dictionary_page_size_limit__ = None; let mut data_page_row_count_limit__ = None; let mut max_row_group_size__ = None; @@ -6081,12 +6072,6 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { } skip_arrow_metadata__ = Some(map_.next_value()?); } - GeneratedField::CacheMetadata => { - if cache_metadata__.is_some() { - return Err(serde::de::Error::duplicate_field("cacheMetadata")); - } - cache_metadata__ = Some(map_.next_value()?); - } GeneratedField::DictionaryPageSizeLimit => { if dictionary_page_size_limit__.is_some() { return Err(serde::de::Error::duplicate_field("dictionaryPageSizeLimit")); @@ -6196,7 +6181,6 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { schema_force_view_types: schema_force_view_types__.unwrap_or_default(), binary_as_string: binary_as_string__.unwrap_or_default(), skip_arrow_metadata: skip_arrow_metadata__.unwrap_or_default(), - cache_metadata: cache_metadata__.unwrap_or_default(), dictionary_page_size_limit: dictionary_page_size_limit__.unwrap_or_default(), data_page_row_count_limit: data_page_row_count_limit__.unwrap_or_default(), max_row_group_size: max_row_group_size__.unwrap_or_default(), diff --git a/datafusion/proto-common/src/generated/prost.rs b/datafusion/proto-common/src/generated/prost.rs index da8d006f62eb..aa23cea57470 100644 --- a/datafusion/proto-common/src/generated/prost.rs +++ b/datafusion/proto-common/src/generated/prost.rs @@ -795,9 +795,6 @@ pub struct ParquetOptions { /// default = false #[prost(bool, tag = "30")] pub skip_arrow_metadata: bool, - /// default = false - #[prost(bool, tag = "33")] - pub cache_metadata: bool, #[prost(uint64, tag = "12")] pub dictionary_page_size_limit: u64, #[prost(uint64, tag = "18")] diff --git a/datafusion/proto-common/src/to_proto/mod.rs b/datafusion/proto-common/src/to_proto/mod.rs index 0f0204b3a6b0..c06427065733 100644 --- a/datafusion/proto-common/src/to_proto/mod.rs +++ b/datafusion/proto-common/src/to_proto/mod.rs @@ -842,7 +842,6 @@ impl TryFrom<&ParquetOptions> for protobuf::ParquetOptions { binary_as_string: value.binary_as_string, skip_arrow_metadata: value.skip_arrow_metadata, coerce_int96_opt: value.coerce_int96.clone().map(protobuf::parquet_options::CoerceInt96Opt::CoerceInt96), - cache_metadata: value.cache_metadata, }) } } diff --git a/datafusion/proto/src/generated/datafusion_proto_common.rs b/datafusion/proto/src/generated/datafusion_proto_common.rs index da8d006f62eb..aa23cea57470 100644 --- a/datafusion/proto/src/generated/datafusion_proto_common.rs +++ b/datafusion/proto/src/generated/datafusion_proto_common.rs @@ -795,9 +795,6 @@ pub struct ParquetOptions { /// default = false #[prost(bool, tag = "30")] pub skip_arrow_metadata: bool, - /// default = false - #[prost(bool, tag = "33")] - pub cache_metadata: bool, #[prost(uint64, tag = "12")] pub dictionary_page_size_limit: u64, #[prost(uint64, tag = "18")] diff --git a/datafusion/proto/src/logical_plan/file_formats.rs b/datafusion/proto/src/logical_plan/file_formats.rs index 139d615e9895..654607bd733d 100644 --- a/datafusion/proto/src/logical_plan/file_formats.rs +++ b/datafusion/proto/src/logical_plan/file_formats.rs @@ -411,7 +411,6 @@ impl TableParquetOptionsProto { coerce_int96_opt: global_options.global.coerce_int96.map(|compression| { parquet_options::CoerceInt96Opt::CoerceInt96(compression) }), - cache_metadata: global_options.global.cache_metadata, }), column_specific_options: column_specific_options.into_iter().map(|(column_name, options)| { ParquetColumnSpecificOptions { @@ -505,7 +504,6 @@ impl From<&ParquetOptionsProto> for ParquetOptions { coerce_int96: proto.coerce_int96_opt.as_ref().map(|opt| match opt { parquet_options::CoerceInt96Opt::CoerceInt96(coerce_int96) => coerce_int96.clone(), }), - cache_metadata: proto.cache_metadata, } } } diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index 2cdfff4400b3..c87abb972ea6 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -231,7 +231,6 @@ datafusion.execution.parquet.bloom_filter_fpp NULL datafusion.execution.parquet.bloom_filter_ndv NULL datafusion.execution.parquet.bloom_filter_on_read true datafusion.execution.parquet.bloom_filter_on_write false -datafusion.execution.parquet.cache_metadata false datafusion.execution.parquet.coerce_int96 NULL datafusion.execution.parquet.column_index_truncate_length 64 datafusion.execution.parquet.compression zstd(3) @@ -345,7 +344,6 @@ datafusion.execution.parquet.bloom_filter_fpp NULL (writing) Sets bloom filter f datafusion.execution.parquet.bloom_filter_ndv NULL (writing) Sets bloom filter number of distinct values. If NULL, uses default parquet writer setting datafusion.execution.parquet.bloom_filter_on_read true (reading) Use any available bloom filters when reading parquet files datafusion.execution.parquet.bloom_filter_on_write false (writing) Write bloom filters for all columns when creating parquet files -datafusion.execution.parquet.cache_metadata false (reading) Whether or not to enable the caching of embedded metadata of Parquet files (footer and page metadata). Enabling it can offer substantial performance improvements for repeated queries over large files. By default, the cache is automatically invalidated when the underlying file is modified. datafusion.execution.parquet.coerce_int96 NULL (reading) If true, parquet reader will read columns of physical type int96 as originating from a different resolution than nanosecond. This is useful for reading data from systems like Spark which stores microsecond resolution timestamps in an int96 allowing it to write values with a larger date range than 64-bit timestamps with nanosecond resolution. datafusion.execution.parquet.column_index_truncate_length 64 (writing) Sets column index truncate length datafusion.execution.parquet.compression zstd(3) (writing) Sets default parquet compression codec. Valid values are: uncompressed, snappy, gzip(level), lzo, brotli(level), lz4, zstd(level), and lz4_raw. These values are not case sensitive. If NULL, uses default parquet writer setting Note that this default setting is not the same as the default parquet writer setting. diff --git a/datafusion/sqllogictest/test_files/parquet.slt b/datafusion/sqllogictest/test_files/parquet.slt index 0beb2e8f5d20..ef99423c2797 100644 --- a/datafusion/sqllogictest/test_files/parquet.slt +++ b/datafusion/sqllogictest/test_files/parquet.slt @@ -766,10 +766,6 @@ OPTIONS (MAX_ROW_GROUP_SIZE 4096, DATA_PAGE_ROW_COUNT_LIMIT 2048); ---- 20000 -# Enable the cache -statement ok -set datafusion.execution.parquet.cache_metadata = true; - statement ok CREATE EXTERNAL TABLE t STORED AS PARQUET @@ -867,5 +863,3 @@ select part, k, v from t order by k statement ok DROP TABLE t; -statement ok -set datafusion.execution.parquet.cache_metadata = false; diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 3fc8e984375d..62bacf27cae3 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -87,8 +87,7 @@ The following configuration settings are available: | datafusion.execution.parquet.schema_force_view_types | true | (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. | | datafusion.execution.parquet.binary_as_string | false | (reading) If true, parquet reader will read columns of `Binary/LargeBinary` with `Utf8`, and `BinaryView` with `Utf8View`. Parquet files generated by some legacy writers do not correctly set the UTF8 flag for strings, causing string columns to be loaded as BLOB instead. | | datafusion.execution.parquet.coerce_int96 | NULL | (reading) If true, parquet reader will read columns of physical type int96 as originating from a different resolution than nanosecond. This is useful for reading data from systems like Spark which stores microsecond resolution timestamps in an int96 allowing it to write values with a larger date range than 64-bit timestamps with nanosecond resolution. | -| datafusion.execution.parquet.bloom_filter_on_read | true | (reading) Use any available bloom filters when reading parquet files | -| datafusion.execution.parquet.cache_metadata | false | (reading) Whether or not to enable the caching of embedded metadata of Parquet files (footer and page metadata). Enabling it can offer substantial performance improvements for repeated queries over large files. By default, the cache is automatically invalidated when the underlying file is modified. | +| datafusion.execution.parquet.bloom_filter_on_read | true | (reading) Use any available bloom filters when reading parquet files | | | datafusion.execution.parquet.data_pagesize_limit | 1048576 | (writing) Sets best effort maximum size of data page in bytes | | datafusion.execution.parquet.write_batch_size | 1024 | (writing) Sets write_batch_size in bytes | | datafusion.execution.parquet.writer_version | 1.0 | (writing) Sets parquet writer version valid values are "1.0" and "2.0" | From dae15bb7355231f66aa73428f05a7898ca492acc Mon Sep 17 00:00:00 2001 From: Jonathan Date: Thu, 7 Aug 2025 03:26:39 -0400 Subject: [PATCH 2/7] prettier --- docs/source/library-user-guide/query-optimizer.md | 2 -- docs/source/user-guide/concepts-readings-events.md | 1 - docs/source/user-guide/configs.md | 4 ++-- docs/source/user-guide/sql/scalar_functions.md | 3 --- 4 files changed, 2 insertions(+), 8 deletions(-) diff --git a/docs/source/library-user-guide/query-optimizer.md b/docs/source/library-user-guide/query-optimizer.md index 224510083f9f..8fc6acde7e12 100644 --- a/docs/source/library-user-guide/query-optimizer.md +++ b/docs/source/library-user-guide/query-optimizer.md @@ -440,13 +440,11 @@ When analyzing expressions, DataFusion runs boundary analysis using interval ari Consider a simple predicate like age > 18 AND age <= 25. The analysis flows as follows: 1. Context Initialization - - Begin with known column statistics - Set up initial boundaries based on column constraints - Initialize the shared analysis context 2. Expression Tree Walk - - Analyze each node in the expression tree - Propagate boundary information upward - Allow child nodes to influence parent boundaries diff --git a/docs/source/user-guide/concepts-readings-events.md b/docs/source/user-guide/concepts-readings-events.md index ad444ef91c47..3b5a244f04ca 100644 --- a/docs/source/user-guide/concepts-readings-events.md +++ b/docs/source/user-guide/concepts-readings-events.md @@ -70,7 +70,6 @@ This is a list of DataFusion related blog posts, articles, and other resources. - **2024-10-16** [Blog: Candle Image Segmentation](https://www.letsql.com/posts/candle-image-segmentation/) - **2024-09-23 → 2024-12-02** [Talks: Carnegie Mellon University: Database Building Blocks Seminar Series - Fall 2024](https://db.cs.cmu.edu/seminar2024/) - - **2024-11-12** [Video: Building InfluxDB 3.0 with the FDAP Stack: Apache Flight, DataFusion, Arrow and Parquet (Paul Dix)](https://www.youtube.com/watch?v=AGS4GNGDK_4) - **2024-11-04** [Video: Synnada: Towards “Unified” Compute Engines: Opportunities and Challenges (Mehmet Ozan Kabak)](https://www.youtube.com/watch?v=z38WY9uZtt4) diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 62bacf27cae3..e08c08aa4d4a 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -64,7 +64,7 @@ SET datafusion.execution.target_partitions = '1'; The following configuration settings are available: | key | default | description | -| ----------------------------------------------------------------------- | ------------------------- | ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | +| ----------------------------------------------------------------------- | ------------------------- | ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | --- | | datafusion.catalog.create_default_catalog_and_schema | true | Whether the default catalog and schema should be created automatically. | | datafusion.catalog.default_catalog | datafusion | The default catalog name - this impacts what SQL queries use if not specified | | datafusion.catalog.default_schema | public | The default schema name - this impacts what SQL queries use if not specified | @@ -87,7 +87,7 @@ The following configuration settings are available: | datafusion.execution.parquet.schema_force_view_types | true | (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. | | datafusion.execution.parquet.binary_as_string | false | (reading) If true, parquet reader will read columns of `Binary/LargeBinary` with `Utf8`, and `BinaryView` with `Utf8View`. Parquet files generated by some legacy writers do not correctly set the UTF8 flag for strings, causing string columns to be loaded as BLOB instead. | | datafusion.execution.parquet.coerce_int96 | NULL | (reading) If true, parquet reader will read columns of physical type int96 as originating from a different resolution than nanosecond. This is useful for reading data from systems like Spark which stores microsecond resolution timestamps in an int96 allowing it to write values with a larger date range than 64-bit timestamps with nanosecond resolution. | -| datafusion.execution.parquet.bloom_filter_on_read | true | (reading) Use any available bloom filters when reading parquet files | | +| datafusion.execution.parquet.bloom_filter_on_read | true | (reading) Use any available bloom filters when reading parquet files | | | datafusion.execution.parquet.data_pagesize_limit | 1048576 | (writing) Sets best effort maximum size of data page in bytes | | datafusion.execution.parquet.write_batch_size | 1024 | (writing) Sets write_batch_size in bytes | | datafusion.execution.parquet.writer_version | 1.0 | (writing) Sets parquet writer version valid values are "1.0" and "2.0" | diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index d49fc22dabb4..a76aaa51a251 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -2048,7 +2048,6 @@ date_bin(interval, expression, origin-timestamp) - **interval**: Bin interval. - **expression**: Time expression to operate on. Can be a constant, column, or function. - **origin-timestamp**: Optional. Starting point used to determine bin boundaries. If not specified defaults 1970-01-01T00:00:00Z (the UNIX epoch in UTC). The following intervals are supported: - - nanoseconds - microseconds - milliseconds @@ -2102,7 +2101,6 @@ date_part(part, expression) #### Arguments - **part**: Part of the date to return. The following date parts are supported: - - year - quarter (emits value in inclusive range [1, 4] based on which quartile of the year the date is in) - month @@ -2141,7 +2139,6 @@ date_trunc(precision, expression) #### Arguments - **precision**: Time precision to truncate to. The following precisions are supported: - - year / YEAR - quarter / QUARTER - month / MONTH From ff2c726b3880f88bb8c4c5471e683b61b124660b Mon Sep 17 00:00:00 2001 From: Jonathan Date: Thu, 7 Aug 2025 03:30:58 -0400 Subject: [PATCH 3/7] fix prettier? --- docs/source/library-user-guide/query-optimizer.md | 2 ++ docs/source/user-guide/concepts-readings-events.md | 1 + docs/source/user-guide/sql/scalar_functions.md | 3 +++ 3 files changed, 6 insertions(+) diff --git a/docs/source/library-user-guide/query-optimizer.md b/docs/source/library-user-guide/query-optimizer.md index 8fc6acde7e12..224510083f9f 100644 --- a/docs/source/library-user-guide/query-optimizer.md +++ b/docs/source/library-user-guide/query-optimizer.md @@ -440,11 +440,13 @@ When analyzing expressions, DataFusion runs boundary analysis using interval ari Consider a simple predicate like age > 18 AND age <= 25. The analysis flows as follows: 1. Context Initialization + - Begin with known column statistics - Set up initial boundaries based on column constraints - Initialize the shared analysis context 2. Expression Tree Walk + - Analyze each node in the expression tree - Propagate boundary information upward - Allow child nodes to influence parent boundaries diff --git a/docs/source/user-guide/concepts-readings-events.md b/docs/source/user-guide/concepts-readings-events.md index 3b5a244f04ca..ad444ef91c47 100644 --- a/docs/source/user-guide/concepts-readings-events.md +++ b/docs/source/user-guide/concepts-readings-events.md @@ -70,6 +70,7 @@ This is a list of DataFusion related blog posts, articles, and other resources. - **2024-10-16** [Blog: Candle Image Segmentation](https://www.letsql.com/posts/candle-image-segmentation/) - **2024-09-23 → 2024-12-02** [Talks: Carnegie Mellon University: Database Building Blocks Seminar Series - Fall 2024](https://db.cs.cmu.edu/seminar2024/) + - **2024-11-12** [Video: Building InfluxDB 3.0 with the FDAP Stack: Apache Flight, DataFusion, Arrow and Parquet (Paul Dix)](https://www.youtube.com/watch?v=AGS4GNGDK_4) - **2024-11-04** [Video: Synnada: Towards “Unified” Compute Engines: Opportunities and Challenges (Mehmet Ozan Kabak)](https://www.youtube.com/watch?v=z38WY9uZtt4) diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index a76aaa51a251..d49fc22dabb4 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -2048,6 +2048,7 @@ date_bin(interval, expression, origin-timestamp) - **interval**: Bin interval. - **expression**: Time expression to operate on. Can be a constant, column, or function. - **origin-timestamp**: Optional. Starting point used to determine bin boundaries. If not specified defaults 1970-01-01T00:00:00Z (the UNIX epoch in UTC). The following intervals are supported: + - nanoseconds - microseconds - milliseconds @@ -2101,6 +2102,7 @@ date_part(part, expression) #### Arguments - **part**: Part of the date to return. The following date parts are supported: + - year - quarter (emits value in inclusive range [1, 4] based on which quartile of the year the date is in) - month @@ -2139,6 +2141,7 @@ date_trunc(precision, expression) #### Arguments - **precision**: Time precision to truncate to. The following precisions are supported: + - year / YEAR - quarter / QUARTER - month / MONTH From a6c53d6a8c5971e10e227121b980eb20104c6857 Mon Sep 17 00:00:00 2001 From: Jonathan Date: Thu, 7 Aug 2025 03:37:47 -0400 Subject: [PATCH 4/7] fix --- datafusion/proto-common/src/generated/pbjson.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/datafusion/proto-common/src/generated/pbjson.rs b/datafusion/proto-common/src/generated/pbjson.rs index 61001ae3169e..48782ff1d93a 100644 --- a/datafusion/proto-common/src/generated/pbjson.rs +++ b/datafusion/proto-common/src/generated/pbjson.rs @@ -5804,7 +5804,6 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { "binaryAsString", "skip_arrow_metadata", "skipArrowMetadata", - "cacheMetadata", "dictionary_page_size_limit", "dictionaryPageSizeLimit", "data_page_row_count_limit", @@ -5851,7 +5850,6 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { SchemaForceViewTypes, BinaryAsString, SkipArrowMetadata, - CacheMetadata, DictionaryPageSizeLimit, DataPageRowCountLimit, MaxRowGroupSize, From df29cd75692cd83bdd36e6dd436491e80282f2ac Mon Sep 17 00:00:00 2001 From: Jonathan Date: Thu, 7 Aug 2025 04:02:47 -0400 Subject: [PATCH 5/7] fix config --- docs/source/user-guide/configs.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index e08c08aa4d4a..7868db9419d8 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -64,7 +64,7 @@ SET datafusion.execution.target_partitions = '1'; The following configuration settings are available: | key | default | description | -| ----------------------------------------------------------------------- | ------------------------- | ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | --- | +| ----------------------------------------------------------------------- | ------------------------- | ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | | datafusion.catalog.create_default_catalog_and_schema | true | Whether the default catalog and schema should be created automatically. | | datafusion.catalog.default_catalog | datafusion | The default catalog name - this impacts what SQL queries use if not specified | | datafusion.catalog.default_schema | public | The default schema name - this impacts what SQL queries use if not specified | @@ -87,7 +87,7 @@ The following configuration settings are available: | datafusion.execution.parquet.schema_force_view_types | true | (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. | | datafusion.execution.parquet.binary_as_string | false | (reading) If true, parquet reader will read columns of `Binary/LargeBinary` with `Utf8`, and `BinaryView` with `Utf8View`. Parquet files generated by some legacy writers do not correctly set the UTF8 flag for strings, causing string columns to be loaded as BLOB instead. | | datafusion.execution.parquet.coerce_int96 | NULL | (reading) If true, parquet reader will read columns of physical type int96 as originating from a different resolution than nanosecond. This is useful for reading data from systems like Spark which stores microsecond resolution timestamps in an int96 allowing it to write values with a larger date range than 64-bit timestamps with nanosecond resolution. | -| datafusion.execution.parquet.bloom_filter_on_read | true | (reading) Use any available bloom filters when reading parquet files | | +| datafusion.execution.parquet.bloom_filter_on_read | true | (reading) Use any available bloom filters when reading parquet files | | datafusion.execution.parquet.data_pagesize_limit | 1048576 | (writing) Sets best effort maximum size of data page in bytes | | datafusion.execution.parquet.write_batch_size | 1024 | (writing) Sets write_batch_size in bytes | | datafusion.execution.parquet.writer_version | 1.0 | (writing) Sets parquet writer version valid values are "1.0" and "2.0" | From ed628f80304ca6b18e79e1d41dfb51ab26426bf1 Mon Sep 17 00:00:00 2001 From: Jonathan Date: Thu, 7 Aug 2025 05:04:42 -0400 Subject: [PATCH 6/7] fix test behaviour --- datafusion/core/tests/parquet/page_pruning.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/core/tests/parquet/page_pruning.rs b/datafusion/core/tests/parquet/page_pruning.rs index 5b37c55c09e4..f851c199eb4d 100644 --- a/datafusion/core/tests/parquet/page_pruning.rs +++ b/datafusion/core/tests/parquet/page_pruning.rs @@ -903,8 +903,8 @@ async fn without_pushdown_filter() { ) .unwrap(); - // Without filter will not read pageIndex. - assert!(bytes_scanned_with_filter > bytes_scanned_without_filter); + // Same amount of bytes are scanned when defaulting to cache parquet metadata + assert!(bytes_scanned_with_filter == bytes_scanned_without_filter); } #[tokio::test] From 0338633503e25e2e0d015e575bb5941e9ff20783 Mon Sep 17 00:00:00 2001 From: Jonathan Date: Thu, 7 Aug 2025 07:15:15 -0400 Subject: [PATCH 7/7] fix --- .../datasource-parquet/src/file_format.rs | 17 +++++++---------- 1 file changed, 7 insertions(+), 10 deletions(-) diff --git a/datafusion/datasource-parquet/src/file_format.rs b/datafusion/datasource-parquet/src/file_format.rs index 6099fd359486..8276a3a8ff3c 100644 --- a/datafusion/datasource-parquet/src/file_format.rs +++ b/datafusion/datasource-parquet/src/file_format.rs @@ -448,16 +448,13 @@ impl FileFormat for ParquetFormat { let mut source = ParquetSource::new(self.options.clone()); // Use the CachedParquetFileReaderFactory - if state.runtime_env().cache_manager.get_file_metadata_cache() { - let metadata_cache = - state.runtime_env().cache_manager.get_file_metadata_cache(); - let store = state - .runtime_env() - .object_store(conf.object_store_url.clone())?; - let cached_parquet_read_factory = - Arc::new(CachedParquetFileReaderFactory::new(store, metadata_cache)); - source = source.with_parquet_file_reader_factory(cached_parquet_read_factory); - } + let metadata_cache = state.runtime_env().cache_manager.get_file_metadata_cache(); + let store = state + .runtime_env() + .object_store(conf.object_store_url.clone())?; + let cached_parquet_read_factory = + Arc::new(CachedParquetFileReaderFactory::new(store, metadata_cache)); + source = source.with_parquet_file_reader_factory(cached_parquet_read_factory); if let Some(metadata_size_hint) = metadata_size_hint { source = source.with_metadata_size_hint(metadata_size_hint)