Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
10 changes: 10 additions & 0 deletions datafusion/common/src/config.rs
Original file line number Diff line number Diff line change
Expand Up @@ -908,6 +908,16 @@ config_namespace! {
/// nanosecond resolution.
pub coerce_int96: Option<String>, transform = str::to_lowercase, default = None

/// (reading) Optional timezone applied to INT96 columns when `coerce_int96`
/// is set. When `Some`, INT96 columns coerce to
/// `Timestamp(<coerce_int96>, Some(<tz>))` instead of the default
/// `Timestamp(<coerce_int96>, None)`. Spark and other systems write INT96
/// values as UTC-adjusted instants, so callers that need the resulting
/// Arrow type to be timezone-aware (e.g. for Spark `TimestampType`
/// semantics) should set this to `"UTC"`. No effect when `coerce_int96`
/// is `None`.
pub coerce_int96_tz: Option<String>, default = None

/// (reading) Use any available bloom filters when reading parquet files
pub bloom_filter_on_read: bool, default = true

Expand Down
3 changes: 3 additions & 0 deletions datafusion/common/src/file_options/parquet_writer.rs
Original file line number Diff line number Diff line change
Expand Up @@ -208,6 +208,7 @@ impl ParquetOptions {
schema_force_view_types: _,
binary_as_string: _, // not used for writer props
coerce_int96: _, // not used for writer props
coerce_int96_tz: _, // not used for writer props
skip_arrow_metadata: _,
max_predicate_cache_size: _,
} = self;
Expand Down Expand Up @@ -482,6 +483,7 @@ mod tests {
binary_as_string: defaults.binary_as_string,
skip_arrow_metadata: defaults.skip_arrow_metadata,
coerce_int96: None,
coerce_int96_tz: None,
max_predicate_cache_size: defaults.max_predicate_cache_size,
use_content_defined_chunking: defaults.use_content_defined_chunking.clone(),
}
Expand Down Expand Up @@ -600,6 +602,7 @@ mod tests {
binary_as_string: global_options_defaults.binary_as_string,
skip_arrow_metadata: global_options_defaults.skip_arrow_metadata,
coerce_int96: None,
coerce_int96_tz: None,
use_content_defined_chunking: props.content_defined_chunking().map(|c| {
CdcOptions {
min_chunk_size: c.min_chunk_size,
Expand Down
14 changes: 12 additions & 2 deletions datafusion/datasource-parquet/src/file_format.rs
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ use std::sync::Arc;
// Re-export so the historical `file_format::*` paths still resolve.
#[expect(deprecated)]
pub use crate::schema_coercion::{
apply_file_schema_type_coercions, coerce_file_schema_to_string_type,
Int96Coercer, apply_file_schema_type_coercions, coerce_file_schema_to_string_type,
coerce_file_schema_to_view_type, coerce_int96_to_resolution,
transform_binary_to_string, transform_schema_to_view,
};
Expand Down Expand Up @@ -56,7 +56,9 @@ use datafusion_session::Session;

use crate::metadata::{DFParquetMetadata, lex_ordering_to_sorting_columns};
use crate::reader::CachedParquetFileReaderFactory;
use crate::source::{ParquetSource, parse_coerce_int96_string};
use crate::source::{
ParquetSource, parse_coerce_int96_string, parse_coerce_int96_tz_string,
};
use async_trait::async_trait;
use bytes::Bytes;
use datafusion_datasource::source::DataSourceExec;
Expand Down Expand Up @@ -333,6 +335,13 @@ impl FileFormat for ParquetFormat {
Some(time_unit) => Some(parse_coerce_int96_string(time_unit.as_str())?),
None => None,
};
let coerce_int96_tz = self
.options
.global
.coerce_int96_tz
.as_ref()
.map(|tz| parse_coerce_int96_tz_string(tz))
.transpose()?;

let file_metadata_cache =
state.runtime_env().cache_manager.get_file_metadata_cache();
Expand All @@ -350,6 +359,7 @@ impl FileFormat for ParquetFormat {
.with_decryption_properties(file_decryption_properties)
.with_file_metadata_cache(Some(Arc::clone(&file_metadata_cache)))
.with_coerce_int96(coerce_int96)
.with_coerce_int96_tz(coerce_int96_tz.clone())
.fetch_schema_with_location()
.await?;
Ok::<_, DataFusionError>(result)
Expand Down
19 changes: 13 additions & 6 deletions datafusion/datasource-parquet/src/metadata.rs
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
//! [`DFParquetMetadata`] for fetching Parquet file metadata, statistics
//! and schema information.

use crate::{apply_file_schema_type_coercions, coerce_int96_to_resolution};
use crate::{Int96Coercer, apply_file_schema_type_coercions};
use arrow::array::{Array, ArrayRef, BooleanArray};
use arrow::compute::and;
use arrow::compute::kernels::cmp::eq;
Expand Down Expand Up @@ -72,6 +72,8 @@ pub struct DFParquetMetadata<'a> {
file_metadata_cache: Option<Arc<dyn FileMetadataCache>>,
/// timeunit to coerce INT96 timestamps to
pub coerce_int96: Option<TimeUnit>,
/// Optional timezone applied to INT96-coerced timestamps.
pub coerce_int96_tz: Option<Arc<str>>,
}

impl<'a> DFParquetMetadata<'a> {
Expand All @@ -83,6 +85,7 @@ impl<'a> DFParquetMetadata<'a> {
decryption_properties: None,
file_metadata_cache: None,
coerce_int96: None,
coerce_int96_tz: None,
}
}

Expand Down Expand Up @@ -116,6 +119,12 @@ impl<'a> DFParquetMetadata<'a> {
self
}

/// Set the optional timezone applied to INT96-coerced timestamps.
pub fn with_coerce_int96_tz(mut self, timezone: Option<Arc<str>>) -> Self {
self.coerce_int96_tz = timezone;
self
}

/// Fetch parquet metadata from the remote object store
pub async fn fetch_metadata(&self) -> Result<Arc<ParquetMetaData>> {
// implementation to fetch parquet metadata
Expand Down Expand Up @@ -218,11 +227,9 @@ impl<'a> DFParquetMetadata<'a> {
.coerce_int96
.as_ref()
.and_then(|time_unit| {
coerce_int96_to_resolution(
file_metadata.schema_descr(),
&schema,
time_unit,
)
Int96Coercer::new(file_metadata.schema_descr(), &schema, time_unit)
.with_timezone(self.coerce_int96_tz.clone())
.coerce()
})
.unwrap_or(schema);
Ok(schema)
Expand Down
2 changes: 1 addition & 1 deletion datafusion/datasource-parquet/src/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,7 @@ pub use row_filter::can_expr_be_pushed_down_with_schemas;
pub use row_group_filter::RowGroupAccessPlanFilter;
#[expect(deprecated)]
pub use schema_coercion::{
apply_file_schema_type_coercions, coerce_file_schema_to_string_type,
Int96Coercer, apply_file_schema_type_coercions, coerce_file_schema_to_string_type,
coerce_file_schema_to_view_type, coerce_int96_to_resolution,
transform_binary_to_string, transform_schema_to_view,
};
Expand Down
18 changes: 15 additions & 3 deletions datafusion/datasource-parquet/src/opener/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -29,8 +29,8 @@ use crate::push_decoder::{DecoderBuilderConfig, PushDecoderStreamState};
use crate::row_filter::{RowFilterGenerator, build_projection_read_plan};
use crate::row_group_filter::{BloomFilterStatistics, RowGroupAccessPlanFilter};
use crate::{
ParquetAccessPlan, ParquetFileMetrics, ParquetFileReaderFactory,
apply_file_schema_type_coercions, coerce_int96_to_resolution,
Int96Coercer, ParquetAccessPlan, ParquetFileMetrics, ParquetFileReaderFactory,
apply_file_schema_type_coercions,
};
use arrow::array::RecordBatch;
use arrow::datatypes::DataType;
Expand Down Expand Up @@ -121,6 +121,10 @@ pub(super) struct ParquetMorselizer {
pub enable_row_group_stats_pruning: bool,
/// Coerce INT96 timestamps to specific TimeUnit
pub coerce_int96: Option<TimeUnit>,
/// Optional timezone applied to INT96-coerced timestamps. When `Some`, the
/// coerced column type becomes `Timestamp(<coerce_int96>, Some(<tz>))`.
/// No effect when `coerce_int96` is `None`.
pub coerce_int96_tz: Option<Arc<str>>,
/// Optional parquet FileDecryptionProperties
#[cfg(feature = "parquet_encryption")]
pub file_decryption_properties: Option<Arc<FileDecryptionProperties>>,
Expand Down Expand Up @@ -283,6 +287,7 @@ struct PreparedParquetOpen {
enable_row_group_stats_pruning: bool,
limit: Option<usize>,
coerce_int96: Option<TimeUnit>,
coerce_int96_tz: Option<Arc<str>>,
expr_adapter_factory: Arc<dyn PhysicalExprAdapterFactory>,
predicate_creation_errors: Count,
max_predicate_cache_size: Option<usize>,
Expand Down Expand Up @@ -653,6 +658,7 @@ impl ParquetMorselizer {
enable_row_group_stats_pruning: self.enable_row_group_stats_pruning,
limit: self.limit,
coerce_int96: self.coerce_int96,
coerce_int96_tz: self.coerce_int96_tz.clone(),
expr_adapter_factory: Arc::clone(&self.expr_adapter_factory),
predicate_creation_errors,
max_predicate_cache_size: self.max_predicate_cache_size,
Expand Down Expand Up @@ -780,11 +786,13 @@ impl MetadataLoadedParquetOpen {
}

if let Some(ref coerce) = prepared.coerce_int96
&& let Some(merged) = coerce_int96_to_resolution(
&& let Some(merged) = Int96Coercer::new(
reader_metadata.parquet_schema(),
&physical_file_schema,
coerce,
)
.with_timezone(prepared.coerce_int96_tz.clone())
.coerce()
{
physical_file_schema = Arc::new(merged);
options = options.with_schema(Arc::clone(&physical_file_schema));
Expand Down Expand Up @@ -1603,6 +1611,10 @@ mod test {
enable_bloom_filter: self.enable_bloom_filter,
enable_row_group_stats_pruning: self.enable_row_group_stats_pruning,
coerce_int96: self.coerce_int96,
// End-to-end coercion behavior (including timezone) is
// covered by parquet.slt. No opener-level test currently
// needs a non-default value here.
coerce_int96_tz: None,
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

if None is intentional here, it would be good to highlight the reason?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

Good catch — added a comment in 916029d. The test builder has a with_coerce_int96 setter but no with_coerce_int96_tz yet because none of the existing coerce_int96 tests need a timezone (they all expect the legacy Timestamp(_, None) output). The comment notes that and points at adding a setter analogous to with_coerce_int96 if a future test needs one.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

I added a test

#[cfg(feature = "parquet_encryption")]
file_decryption_properties: None,
expr_adapter_factory: Arc::new(DefaultPhysicalExprAdapterFactory),
Expand Down
Loading
Loading