Skip to content

fix: describe Parquet schema with coerce_int96 #15750

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

Merged
merged 3 commits into from
Apr 19, 2025
Merged
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
16 changes: 14 additions & 2 deletions datafusion/datasource-parquet/src/file_format.rs
Original file line number Diff line number Diff line change
Expand Up @@ -61,7 +61,7 @@ use datafusion_physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan};
use datafusion_session::Session;

use crate::can_expr_be_pushed_down_with_schemas;
use crate::source::ParquetSource;
use crate::source::{parse_coerce_int96_string, ParquetSource};
use async_trait::async_trait;
use bytes::Bytes;
use datafusion_datasource::source::DataSourceExec;
Expand Down Expand Up @@ -304,9 +304,10 @@ async fn fetch_schema_with_location(
store: &dyn ObjectStore,
file: &ObjectMeta,
metadata_size_hint: Option<usize>,
coerce_int96: Option<TimeUnit>,
) -> Result<(Path, Schema)> {
let loc_path = file.location.clone();
let schema = fetch_schema(store, file, metadata_size_hint).await?;
let schema = fetch_schema(store, file, metadata_size_hint, coerce_int96).await?;
Ok((loc_path, schema))
}

Expand Down Expand Up @@ -337,12 +338,17 @@ impl FileFormat for ParquetFormat {
store: &Arc<dyn ObjectStore>,
objects: &[ObjectMeta],
) -> Result<SchemaRef> {
let coerce_int96 = match self.coerce_int96() {
Some(time_unit) => Some(parse_coerce_int96_string(time_unit.as_str())?),
None => None,
};
let mut schemas: Vec<_> = futures::stream::iter(objects)
.map(|object| {
fetch_schema_with_location(
store.as_ref(),
object,
self.metadata_size_hint(),
coerce_int96,
)
})
.boxed() // Workaround https://github.com/rust-lang/rust/issues/64552
Expand Down Expand Up @@ -825,13 +831,19 @@ async fn fetch_schema(
store: &dyn ObjectStore,
file: &ObjectMeta,
metadata_size_hint: Option<usize>,
coerce_int96: Option<TimeUnit>,
) -> Result<Schema> {
let metadata = fetch_parquet_metadata(store, file, metadata_size_hint).await?;
let file_metadata = metadata.file_metadata();
let schema = parquet_to_arrow_schema(
file_metadata.schema_descr(),
file_metadata.key_value_metadata(),
)?;
let schema = coerce_int96
.and_then(|time_unit| {
coerce_int96_to_resolution(file_metadata.schema_descr(), &schema, &time_unit)
})
.unwrap_or(schema);
Ok(schema)
}

Expand Down
4 changes: 3 additions & 1 deletion datafusion/datasource-parquet/src/source.rs
Original file line number Diff line number Diff line change
Expand Up @@ -439,7 +439,9 @@ impl ParquetSource {
}

/// Parses datafusion.common.config.ParquetOptions.coerce_int96 String to a arrow_schema.datatype.TimeUnit
fn parse_coerce_int96_string(str_setting: &str) -> datafusion_common::Result<TimeUnit> {
pub(crate) fn parse_coerce_int96_string(
str_setting: &str,
) -> datafusion_common::Result<TimeUnit> {
let str_setting_lower: &str = &str_setting.to_lowercase();

match str_setting_lower {
Expand Down
18 changes: 18 additions & 0 deletions datafusion/sqllogictest/test_files/parquet.slt
Original file line number Diff line number Diff line change
Expand Up @@ -629,3 +629,21 @@ physical_plan

statement ok
drop table foo


statement ok
set datafusion.execution.parquet.coerce_int96 = ms;

statement ok
CREATE EXTERNAL TABLE int96_from_spark
STORED AS PARQUET
LOCATION '../../parquet-testing/data/int96_from_spark.parquet';

# Print schema
query TTT
describe int96_from_spark;
----
a Timestamp(Millisecond, None) YES

statement ok
set datafusion.execution.parquet.coerce_int96 = ns;
Loading