Skip to content
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

feat: Add support for Utf8Type and TimeStamp in Parquet statistics #9129

Closed
wants to merge 7 commits into from
Closed
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
130 changes: 119 additions & 11 deletions datafusion/core/src/datasource/file_format/parquet.rs
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,12 @@

//! [`ParquetFormat`]: Parquet [`FileFormat`] abstractions

use arrow_array::RecordBatch;
use arrow_array::types::Utf8Type;
use arrow_array::{
GenericByteArray, RecordBatch, TimestampMicrosecondArray, TimestampMillisecondArray,
TimestampNanosecondArray, TimestampSecondArray,
};
use arrow_schema::TimeUnit;
use async_trait::async_trait;
use datafusion_common::stats::Precision;
use datafusion_physical_plan::metrics::MetricsSet;
Expand Down Expand Up @@ -333,18 +338,98 @@ fn summarize_min_max(
.unwrap_or_else(|_| min_values[i] = None);
}
}
ParquetStatistics::Int64(s) if DataType::Int64 == *fields[i].data_type() => {
if let Some(max_value) = &mut max_values[i] {
max_value
.update_batch(&[Arc::new(Int64Array::from_value(*s.max(), 1))])
.unwrap_or_else(|_| max_values[i] = None);
ParquetStatistics::Int64(s) => match fields[i].data_type().clone() {
DataType::Int64 => {
if let Some(max_value) = &mut max_values[i] {
max_value
.update_batch(&[Arc::new(Int64Array::from_value(*s.max(), 1))])
.unwrap_or_else(|_| max_values[i] = None);
}
if let Some(min_value) = &mut min_values[i] {
min_value
.update_batch(&[Arc::new(Int64Array::from_value(*s.min(), 1))])
.unwrap_or_else(|_| min_values[i] = None);
}
}
if let Some(min_value) = &mut min_values[i] {
min_value
.update_batch(&[Arc::new(Int64Array::from_value(*s.min(), 1))])
.unwrap_or_else(|_| min_values[i] = None);
DataType::Timestamp(time_unit, time_zone) => match time_unit {
TimeUnit::Second => {
if let Some(max_value) = &mut max_values[i] {
max_value
.update_batch(&[Arc::new(
TimestampSecondArray::from_value(*s.max(), 1)
.with_timezone_opt(time_zone.clone()),
)])
.unwrap_or_else(|_| max_values[i] = None);
}
if let Some(min_value) = &mut min_values[i] {
min_value
.update_batch(&[Arc::new(
TimestampSecondArray::from_value(*s.min(), 1)
.with_timezone_opt(time_zone),
)])
.unwrap_or_else(|_| min_values[i] = None);
}
}
TimeUnit::Millisecond => {
if let Some(max_value) = &mut max_values[i] {
max_value
.update_batch(&[Arc::new(
TimestampMillisecondArray::from_value(*s.max(), 1)
.with_timezone_opt(time_zone.clone()),
)])
.unwrap_or_else(|_| max_values[i] = None);
}
if let Some(min_value) = &mut min_values[i] {
min_value
.update_batch(&[Arc::new(
TimestampMillisecondArray::from_value(*s.min(), 1)
.with_timezone_opt(time_zone),
)])
.unwrap_or_else(|_| min_values[i] = None);
}
}
TimeUnit::Microsecond => {
if let Some(max_value) = &mut max_values[i] {
max_value
.update_batch(&[Arc::new(
TimestampMicrosecondArray::from_value(*s.max(), 1)
.with_timezone_opt(time_zone.clone()),
)])
.unwrap_or_else(|_| max_values[i] = None);
}
if let Some(min_value) = &mut min_values[i] {
min_value
.update_batch(&[Arc::new(
TimestampMicrosecondArray::from_value(*s.min(), 1)
.with_timezone_opt(time_zone),
)])
.unwrap_or_else(|_| min_values[i] = None);
}
}
TimeUnit::Nanosecond => {
if let Some(max_value) = &mut max_values[i] {
max_value
.update_batch(&[Arc::new(
TimestampNanosecondArray::from_value(*s.max(), 1)
.with_timezone_opt(time_zone.clone()),
)])
.unwrap_or_else(|_| max_values[i] = None);
}
if let Some(min_value) = &mut min_values[i] {
min_value
.update_batch(&[Arc::new(
TimestampNanosecondArray::from_value(*s.min(), 1)
.with_timezone_opt(time_zone),
)])
.unwrap_or_else(|_| min_values[i] = None);
}
}
},
_ => {
max_values[i] = None;
min_values[i] = None;
}
}
},
ParquetStatistics::Float(s) if DataType::Float32 == *fields[i].data_type() => {
if let Some(max_value) = &mut max_values[i] {
max_value
Expand All @@ -369,6 +454,29 @@ fn summarize_min_max(
.unwrap_or_else(|_| min_values[i] = None);
}
}

ParquetStatistics::ByteArray(s)
if matches!(fields[i].data_type(), DataType::Utf8 | DataType::LargeUtf8) =>
{
if let Some(max_value) = &mut max_values[i] {
Copy link
Contributor

Choose a reason for hiding this comment

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

I believe byte arrays are also used to store DataType::Decimal values as well (though hopefully if we consolidate the statistics conversion code it will "just work")

let stat_value = s.max().as_utf8().unwrap_or_default();
let array: GenericByteArray<Utf8Type> = vec![Some(stat_value)].into();
if max_value.update_batch(&[Arc::new(array)]).is_err()
|| stat_value.is_empty()
{
max_values[i] = None;
}
}
if let Some(min_value) = &mut min_values[i] {
let stat_value = s.min().as_utf8().unwrap_or_default();
let array: GenericByteArray<Utf8Type> = vec![Some(stat_value)].into();
if min_value.update_batch(&[Arc::new(array)]).is_err()
|| stat_value.is_empty()
{
min_values[i] = None;
}
}
}
_ => {
max_values[i] = None;
min_values[i] = None;
Expand Down
Loading
Loading