From 09fdea8050149a732895debe57c7f8a246d09b91 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 15 Jul 2024 16:37:14 -0400 Subject: [PATCH] Use upstream StatisticsConveter --- datafusion-examples/examples/parquet_index.rs | 6 +- datafusion/core/Cargo.toml | 4 - datafusion/core/benches/parquet_statistic.rs | 287 -- .../src/datasource/file_format/parquet.rs | 7 +- .../datasource/physical_plan/parquet/mod.rs | 2 - .../physical_plan/parquet/page_filter.rs | 36 +- .../physical_plan/parquet/row_groups.rs | 24 +- .../physical_plan/parquet/statistics.rs | 2632 ----------------- .../core/tests/parquet/arrow_statistics.rs | 2178 -------------- datafusion/core/tests/parquet/mod.rs | 1 - 10 files changed, 48 insertions(+), 5129 deletions(-) delete mode 100644 datafusion/core/benches/parquet_statistic.rs delete mode 100644 datafusion/core/src/datasource/physical_plan/parquet/statistics.rs delete mode 100644 datafusion/core/tests/parquet/arrow_statistics.rs diff --git a/datafusion-examples/examples/parquet_index.rs b/datafusion-examples/examples/parquet_index.rs index 668eda047444..50b2b272b3c1 100644 --- a/datafusion-examples/examples/parquet_index.rs +++ b/datafusion-examples/examples/parquet_index.rs @@ -24,13 +24,11 @@ use arrow::util::pretty::pretty_format_batches; use arrow_schema::SchemaRef; use async_trait::async_trait; use datafusion::datasource::listing::PartitionedFile; -use datafusion::datasource::physical_plan::{ - parquet::StatisticsConverter, - {FileScanConfig, ParquetExec}, -}; +use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; use datafusion::datasource::TableProvider; use datafusion::execution::context::SessionState; use datafusion::execution::object_store::ObjectStoreUrl; +use datafusion::parquet::arrow::arrow_reader::statistics::StatisticsConverter; use datafusion::parquet::arrow::{ arrow_reader::ParquetRecordBatchReaderBuilder, ArrowWriter, }; diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index 532ca8fde9e7..b1da3e11839d 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -212,7 +212,3 @@ name = "sort" [[bench]] harness = false name = "topk_aggregate" - -[[bench]] -harness = false -name = "parquet_statistic" diff --git a/datafusion/core/benches/parquet_statistic.rs b/datafusion/core/benches/parquet_statistic.rs deleted file mode 100644 index 3595e8773b07..000000000000 --- a/datafusion/core/benches/parquet_statistic.rs +++ /dev/null @@ -1,287 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -//! Benchmarks of benchmark for extracting arrow statistics from parquet - -use arrow::array::{ArrayRef, DictionaryArray, Float64Array, StringArray, UInt64Array}; -use arrow_array::{Int32Array, Int64Array, RecordBatch}; -use arrow_schema::{ - DataType::{self, *}, - Field, Schema, -}; -use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion}; -use datafusion::datasource::physical_plan::parquet::StatisticsConverter; -use parquet::{ - arrow::arrow_reader::ArrowReaderOptions, file::properties::WriterProperties, -}; -use parquet::{ - arrow::{arrow_reader::ArrowReaderBuilder, ArrowWriter}, - file::properties::EnabledStatistics, -}; -use std::sync::Arc; -use tempfile::NamedTempFile; -#[derive(Debug, Clone)] -enum TestTypes { - UInt64, - Int64, - F64, - String, - Dictionary, -} - -use std::fmt; - -impl fmt::Display for TestTypes { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - match self { - TestTypes::UInt64 => write!(f, "UInt64"), - TestTypes::Int64 => write!(f, "Int64"), - TestTypes::F64 => write!(f, "F64"), - TestTypes::String => write!(f, "String"), - TestTypes::Dictionary => write!(f, "Dictionary(Int32, String)"), - } - } -} - -fn create_parquet_file( - dtype: TestTypes, - row_groups: usize, - data_page_row_count_limit: &Option, -) -> NamedTempFile { - let schema = match dtype { - TestTypes::UInt64 => { - Arc::new(Schema::new(vec![Field::new("col", DataType::UInt64, true)])) - } - TestTypes::Int64 => { - Arc::new(Schema::new(vec![Field::new("col", DataType::Int64, true)])) - } - TestTypes::F64 => Arc::new(Schema::new(vec![Field::new( - "col", - DataType::Float64, - true, - )])), - TestTypes::String => { - Arc::new(Schema::new(vec![Field::new("col", DataType::Utf8, true)])) - } - TestTypes::Dictionary => Arc::new(Schema::new(vec![Field::new( - "col", - DataType::Dictionary(Box::new(Int32), Box::new(Utf8)), - true, - )])), - }; - - let mut props = WriterProperties::builder().set_max_row_group_size(row_groups); - if let Some(limit) = data_page_row_count_limit { - props = props - .set_data_page_row_count_limit(*limit) - .set_statistics_enabled(EnabledStatistics::Page); - }; - let props = props.build(); - - let file = tempfile::Builder::new() - .suffix(".parquet") - .tempfile() - .unwrap(); - let mut writer = - ArrowWriter::try_new(file.reopen().unwrap(), schema.clone(), Some(props)) - .unwrap(); - - for _ in 0..row_groups { - let batch = match dtype { - TestTypes::UInt64 => make_uint64_batch(), - TestTypes::Int64 => make_int64_batch(), - TestTypes::F64 => make_f64_batch(), - TestTypes::String => make_string_batch(), - TestTypes::Dictionary => make_dict_batch(), - }; - if data_page_row_count_limit.is_some() { - // Send batches one at a time. This allows the - // writer to apply the page limit, that is only - // checked on RecordBatch boundaries. - for i in 0..batch.num_rows() { - writer.write(&batch.slice(i, 1)).unwrap(); - } - } else { - writer.write(&batch).unwrap(); - } - } - writer.close().unwrap(); - file -} - -fn make_uint64_batch() -> RecordBatch { - let array: ArrayRef = Arc::new(UInt64Array::from(vec![ - Some(1), - Some(2), - Some(3), - Some(4), - Some(5), - ])); - RecordBatch::try_new( - Arc::new(arrow::datatypes::Schema::new(vec![ - arrow::datatypes::Field::new("col", UInt64, false), - ])), - vec![array], - ) - .unwrap() -} - -fn make_int64_batch() -> RecordBatch { - let array: ArrayRef = Arc::new(Int64Array::from(vec![ - Some(1), - Some(2), - Some(3), - Some(4), - Some(5), - ])); - RecordBatch::try_new( - Arc::new(arrow::datatypes::Schema::new(vec![ - arrow::datatypes::Field::new("col", Int64, false), - ])), - vec![array], - ) - .unwrap() -} - -fn make_f64_batch() -> RecordBatch { - let array: ArrayRef = Arc::new(Float64Array::from(vec![1.0, 2.0, 3.0, 4.0, 5.0])); - RecordBatch::try_new( - Arc::new(arrow::datatypes::Schema::new(vec![ - arrow::datatypes::Field::new("col", Float64, false), - ])), - vec![array], - ) - .unwrap() -} - -fn make_string_batch() -> RecordBatch { - let array: ArrayRef = Arc::new(StringArray::from(vec!["a", "b", "c", "d", "e"])); - RecordBatch::try_new( - Arc::new(arrow::datatypes::Schema::new(vec![ - arrow::datatypes::Field::new("col", Utf8, false), - ])), - vec![array], - ) - .unwrap() -} - -fn make_dict_batch() -> RecordBatch { - let keys = Int32Array::from(vec![0, 1, 2, 3, 4]); - let values = StringArray::from(vec!["a", "b", "c", "d", "e"]); - let array: ArrayRef = - Arc::new(DictionaryArray::try_new(keys, Arc::new(values)).unwrap()); - RecordBatch::try_new( - Arc::new(Schema::new(vec![Field::new( - "col", - Dictionary(Box::new(Int32), Box::new(Utf8)), - false, - )])), - vec![array], - ) - .unwrap() -} - -fn criterion_benchmark(c: &mut Criterion) { - let row_groups = 100; - use TestTypes::*; - let types = vec![Int64, UInt64, F64, String, Dictionary]; - let data_page_row_count_limits = vec![None, Some(1)]; - - for dtype in types { - for data_page_row_count_limit in &data_page_row_count_limits { - let file = - create_parquet_file(dtype.clone(), row_groups, data_page_row_count_limit); - let file = file.reopen().unwrap(); - let options = ArrowReaderOptions::new().with_page_index(true); - let reader = ArrowReaderBuilder::try_new_with_options(file, options).unwrap(); - let metadata = reader.metadata(); - let row_groups = metadata.row_groups(); - let row_group_indices: Vec<_> = (0..row_groups.len()).collect(); - - let statistic_type = if data_page_row_count_limit.is_some() { - "data page" - } else { - "row group" - }; - - let mut group = c.benchmark_group(format!( - "Extract {} statistics for {}", - statistic_type, - dtype.clone() - )); - group.bench_function( - BenchmarkId::new("extract_statistics", dtype.clone()), - |b| { - b.iter(|| { - let converter = StatisticsConverter::try_new( - "col", - reader.schema(), - reader.parquet_schema(), - ) - .unwrap(); - - if data_page_row_count_limit.is_some() { - let column_page_index = reader - .metadata() - .column_index() - .expect("File should have column page indices"); - - let column_offset_index = reader - .metadata() - .offset_index() - .expect("File should have column offset indices"); - - let _ = converter.data_page_mins( - column_page_index, - column_offset_index, - &row_group_indices, - ); - let _ = converter.data_page_maxes( - column_page_index, - column_offset_index, - &row_group_indices, - ); - let _ = converter.data_page_null_counts( - column_page_index, - column_offset_index, - &row_group_indices, - ); - let _ = converter.data_page_row_counts( - column_offset_index, - row_groups, - &row_group_indices, - ); - } else { - let _ = converter.row_group_mins(row_groups.iter()).unwrap(); - let _ = converter.row_group_maxes(row_groups.iter()).unwrap(); - let _ = converter - .row_group_null_counts(row_groups.iter()) - .unwrap(); - let _ = converter - .row_group_row_counts(row_groups.iter()) - .unwrap(); - } - }) - }, - ); - group.finish(); - } - } -} - -criterion_group!(benches, criterion_benchmark); -criterion_main!(benches); diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 6271d8af3786..2eecceb57efc 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -75,12 +75,11 @@ use tokio::io::{AsyncWrite, AsyncWriteExt}; use tokio::sync::mpsc::{self, Receiver, Sender}; use tokio::task::JoinSet; -use crate::datasource::physical_plan::parquet::{ - ParquetExecBuilder, StatisticsConverter, -}; +use crate::datasource::physical_plan::parquet::ParquetExecBuilder; use futures::{StreamExt, TryStreamExt}; use object_store::path::Path; use object_store::{ObjectMeta, ObjectStore}; +use parquet::arrow::arrow_reader::statistics::StatisticsConverter; /// Initial writing buffer size. Note this is just a size hint for efficiency. It /// will grow beyond the set value if needed. @@ -1310,7 +1309,7 @@ mod tests { .map(|i| i.to_string()) .collect(); let coll: Vec<_> = schema - .all_fields() + .flattened_fields() .into_iter() .map(|i| i.name().to_string()) .collect(); diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index 9d5c64719e75..f2e5bde83ee5 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -54,7 +54,6 @@ mod page_filter; mod reader; mod row_filter; mod row_groups; -mod statistics; mod writer; use crate::datasource::schema_adapter::{ @@ -64,7 +63,6 @@ pub use access_plan::{ParquetAccessPlan, RowGroupAccess}; pub use metrics::ParquetFileMetrics; use opener::ParquetOpener; pub use reader::{DefaultParquetFileReaderFactory, ParquetFileReaderFactory}; -pub use statistics::StatisticsConverter; pub use writer::plan_to_parquet; /// Execution plan for reading one or more Parquet files. diff --git a/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs b/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs index 7429ca593820..876adfa72307 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs @@ -17,6 +17,9 @@ //! Contains code to filter entire pages +use crate::datasource::physical_plan::parquet::parquet_to_arrow_decimal_type; +use crate::datasource::physical_plan::parquet::ParquetAccessPlan; +use crate::physical_optimizer::pruning::{PruningPredicate, PruningStatistics}; use arrow::array::{ BooleanArray, Decimal128Array, Float32Array, Float64Array, Int32Array, Int64Array, StringArray, @@ -28,6 +31,7 @@ use datafusion_common::{Result, ScalarValue}; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::{split_conjunction, PhysicalExpr}; use log::{debug, trace}; +use parquet::arrow::parquet_column; use parquet::schema::types::{ColumnDescriptor, SchemaDescriptor}; use parquet::{ arrow::arrow_reader::{RowSelection, RowSelector}, @@ -40,13 +44,6 @@ use parquet::{ use std::collections::HashSet; use std::sync::Arc; -use crate::datasource::physical_plan::parquet::parquet_to_arrow_decimal_type; -use crate::datasource::physical_plan::parquet::statistics::{ - from_bytes_to_i128, parquet_column, -}; -use crate::datasource::physical_plan::parquet::ParquetAccessPlan; -use crate::physical_optimizer::pruning::{PruningPredicate, PruningStatistics}; - use super::metrics::ParquetFileMetrics; /// A [`PagePruningPredicate`] provides the ability to construct a [`RowSelection`] @@ -521,6 +518,31 @@ macro_rules! get_min_max_values_for_page_index { }}; } +// Copy from arrow-rs +// https://github.com/apache/arrow-rs/blob/198af7a3f4aa20f9bd003209d9f04b0f37bb120e/parquet/src/arrow/buffer/bit_util.rs#L54 +// Convert the byte slice to fixed length byte array with the length of N. +fn sign_extend_be(b: &[u8]) -> [u8; N] { + assert!(b.len() <= N, "Array too large, expected less than {N}"); + let is_negative = (b[0] & 128u8) == 128u8; + let mut result = if is_negative { [255u8; N] } else { [0u8; N] }; + for (d, s) in result.iter_mut().skip(N - b.len()).zip(b) { + *d = *s; + } + result +} + +// TEMP: replicate from arrow-rs until we port the page pruning statistics over +// +// The endian of the input bytes array must be big-endian. +// Convert the bytes array to i128. +// The endian of the input bytes array must be big-endian. +fn from_bytes_to_i128(b: &[u8]) -> i128 { + // The bytes array are from parquet file and must be the big-endian. + // The endian is defined by parquet format, and the reference document + // https://github.com/apache/parquet-format/blob/54e53e5d7794d383529dd30746378f19a12afd58/src/main/thrift/parquet.thrift#L66 + i128::from_be_bytes(sign_extend_be::<16>(b)) +} + impl<'a> PruningStatistics for PagesPruningStatistics<'a> { fn min_values(&self, _column: &datafusion_common::Column) -> Option { get_min_max_values_for_page_index!(self, min) diff --git a/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs b/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs index 9bc79805746f..2105e51734b7 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs @@ -15,9 +15,13 @@ // specific language governing permissions and limitations // under the License. +use crate::datasource::listing::FileRange; +use crate::physical_optimizer::pruning::{PruningPredicate, PruningStatistics}; use arrow::{array::ArrayRef, datatypes::Schema}; use arrow_array::BooleanArray; use datafusion_common::{Column, Result, ScalarValue}; +use parquet::arrow::arrow_reader::statistics::StatisticsConverter; +use parquet::arrow::parquet_column; use parquet::basic::Type; use parquet::data_type::Decimal; use parquet::schema::types::SchemaDescriptor; @@ -29,11 +33,7 @@ use parquet::{ use std::collections::{HashMap, HashSet}; use std::sync::Arc; -use crate::datasource::listing::FileRange; -use crate::datasource::physical_plan::parquet::statistics::parquet_column; -use crate::physical_optimizer::pruning::{PruningPredicate, PruningStatistics}; - -use super::{ParquetAccessPlan, ParquetFileMetrics, StatisticsConverter}; +use super::{ParquetAccessPlan, ParquetFileMetrics}; /// Reduces the [`ParquetAccessPlan`] based on row group level metadata. /// @@ -356,20 +356,24 @@ impl<'a> RowGroupPruningStatistics<'a> { &'a self, column: &'b Column, ) -> Result> { - StatisticsConverter::try_new(&column.name, self.arrow_schema, self.parquet_schema) + Ok(StatisticsConverter::try_new( + &column.name, + self.arrow_schema, + self.parquet_schema, + )?) } } impl<'a> PruningStatistics for RowGroupPruningStatistics<'a> { fn min_values(&self, column: &Column) -> Option { self.statistics_converter(column) - .and_then(|c| c.row_group_mins(self.metadata_iter())) + .and_then(|c| Ok(c.row_group_mins(self.metadata_iter())?)) .ok() } fn max_values(&self, column: &Column) -> Option { self.statistics_converter(column) - .and_then(|c| c.row_group_maxes(self.metadata_iter())) + .and_then(|c| Ok(c.row_group_maxes(self.metadata_iter())?)) .ok() } @@ -379,7 +383,7 @@ impl<'a> PruningStatistics for RowGroupPruningStatistics<'a> { fn null_counts(&self, column: &Column) -> Option { self.statistics_converter(column) - .and_then(|c| c.row_group_null_counts(self.metadata_iter())) + .and_then(|c| Ok(c.row_group_null_counts(self.metadata_iter())?)) .ok() .map(|counts| Arc::new(counts) as ArrayRef) } @@ -387,7 +391,7 @@ impl<'a> PruningStatistics for RowGroupPruningStatistics<'a> { fn row_counts(&self, column: &Column) -> Option { // row counts are the same for all columns in a row group self.statistics_converter(column) - .and_then(|c| c.row_group_row_counts(self.metadata_iter())) + .and_then(|c| Ok(c.row_group_row_counts(self.metadata_iter())?)) .ok() .flatten() .map(|counts| Arc::new(counts) as ArrayRef) diff --git a/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs b/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs deleted file mode 100644 index 59369aba57a9..000000000000 --- a/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs +++ /dev/null @@ -1,2632 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -//! [`StatisticsConverter`] to convert statistics in parquet format to arrow [`ArrayRef`]. - -// TODO: potentially move this to arrow-rs: https://github.com/apache/arrow-rs/issues/4328 - -use arrow::array::{ - BooleanBuilder, FixedSizeBinaryBuilder, LargeStringBuilder, StringBuilder, -}; -use arrow::datatypes::i256; -use arrow::{array::ArrayRef, datatypes::DataType}; -use arrow_array::{ - new_empty_array, new_null_array, BinaryArray, BooleanArray, Date32Array, Date64Array, - Decimal128Array, Decimal256Array, Float16Array, Float32Array, Float64Array, - Int16Array, Int32Array, Int64Array, Int8Array, LargeBinaryArray, - Time32MillisecondArray, Time32SecondArray, Time64MicrosecondArray, - Time64NanosecondArray, TimestampMicrosecondArray, TimestampMillisecondArray, - TimestampNanosecondArray, TimestampSecondArray, UInt16Array, UInt32Array, - UInt64Array, UInt8Array, -}; -use arrow_schema::{Field, FieldRef, Schema, TimeUnit}; -use datafusion_common::{internal_datafusion_err, internal_err, plan_err, Result}; -use half::f16; -use parquet::data_type::{ByteArray, FixedLenByteArray}; -use parquet::file::metadata::{ParquetColumnIndex, ParquetOffsetIndex, RowGroupMetaData}; -use parquet::file::page_index::index::{Index, PageIndex}; -use parquet::file::statistics::Statistics as ParquetStatistics; -use parquet::schema::types::SchemaDescriptor; -use paste::paste; -use std::sync::Arc; - -// Convert the bytes array to i128. -// The endian of the input bytes array must be big-endian. -pub(crate) fn from_bytes_to_i128(b: &[u8]) -> i128 { - // The bytes array are from parquet file and must be the big-endian. - // The endian is defined by parquet format, and the reference document - // https://github.com/apache/parquet-format/blob/54e53e5d7794d383529dd30746378f19a12afd58/src/main/thrift/parquet.thrift#L66 - i128::from_be_bytes(sign_extend_be::<16>(b)) -} - -// Convert the bytes array to i256. -// The endian of the input bytes array must be big-endian. -pub(crate) fn from_bytes_to_i256(b: &[u8]) -> i256 { - i256::from_be_bytes(sign_extend_be::<32>(b)) -} - -// Convert the bytes array to f16 -pub(crate) fn from_bytes_to_f16(b: &[u8]) -> Option { - match b { - [low, high] => Some(f16::from_be_bytes([*high, *low])), - _ => None, - } -} - -// Copy from arrow-rs -// https://github.com/apache/arrow-rs/blob/198af7a3f4aa20f9bd003209d9f04b0f37bb120e/parquet/src/arrow/buffer/bit_util.rs#L54 -// Convert the byte slice to fixed length byte array with the length of N. -fn sign_extend_be(b: &[u8]) -> [u8; N] { - assert!(b.len() <= N, "Array too large, expected less than {N}"); - let is_negative = (b[0] & 128u8) == 128u8; - let mut result = if is_negative { [255u8; N] } else { [0u8; N] }; - for (d, s) in result.iter_mut().skip(N - b.len()).zip(b) { - *d = *s; - } - result -} - -/// Define an adapter iterator for extracting statistics from an iterator of -/// `ParquetStatistics` -/// -/// -/// Handles checking if the statistics are present and valid with the correct type. -/// -/// Parameters: -/// * `$iterator_type` is the name of the iterator type (e.g. `MinBooleanStatsIterator`) -/// * `$func` is the function to call to get the value (e.g. `min` or `max`) -/// * `$parquet_statistics_type` is the type of the statistics (e.g. `ParquetStatistics::Boolean`) -/// * `$stat_value_type` is the type of the statistics value (e.g. `bool`) -macro_rules! make_stats_iterator { - ($iterator_type:ident, $func:ident, $parquet_statistics_type:path, $stat_value_type:ty) => { - /// Maps an iterator of `ParquetStatistics` into an iterator of - /// `&$stat_value_type`` - /// - /// Yielded elements: - /// * Some(stats) if valid - /// * None if the statistics are not present, not valid, or not $stat_value_type - struct $iterator_type<'a, I> - where - I: Iterator>, - { - iter: I, - } - - impl<'a, I> $iterator_type<'a, I> - where - I: Iterator>, - { - /// Create a new iterator to extract the statistics - fn new(iter: I) -> Self { - Self { iter } - } - } - - /// Implement the Iterator trait for the iterator - impl<'a, I> Iterator for $iterator_type<'a, I> - where - I: Iterator>, - { - type Item = Option<&'a $stat_value_type>; - - /// return the next statistics value - fn next(&mut self) -> Option { - let next = self.iter.next(); - next.map(|x| { - x.and_then(|stats| match stats { - $parquet_statistics_type(s) if stats.has_min_max_set() => { - Some(s.$func()) - } - _ => None, - }) - }) - } - - fn size_hint(&self) -> (usize, Option) { - self.iter.size_hint() - } - } - }; -} - -make_stats_iterator!( - MinBooleanStatsIterator, - min, - ParquetStatistics::Boolean, - bool -); -make_stats_iterator!( - MaxBooleanStatsIterator, - max, - ParquetStatistics::Boolean, - bool -); -make_stats_iterator!(MinInt32StatsIterator, min, ParquetStatistics::Int32, i32); -make_stats_iterator!(MaxInt32StatsIterator, max, ParquetStatistics::Int32, i32); -make_stats_iterator!(MinInt64StatsIterator, min, ParquetStatistics::Int64, i64); -make_stats_iterator!(MaxInt64StatsIterator, max, ParquetStatistics::Int64, i64); -make_stats_iterator!(MinFloatStatsIterator, min, ParquetStatistics::Float, f32); -make_stats_iterator!(MaxFloatStatsIterator, max, ParquetStatistics::Float, f32); -make_stats_iterator!(MinDoubleStatsIterator, min, ParquetStatistics::Double, f64); -make_stats_iterator!(MaxDoubleStatsIterator, max, ParquetStatistics::Double, f64); -make_stats_iterator!( - MinByteArrayStatsIterator, - min_bytes, - ParquetStatistics::ByteArray, - [u8] -); -make_stats_iterator!( - MaxByteArrayStatsIterator, - max_bytes, - ParquetStatistics::ByteArray, - [u8] -); -make_stats_iterator!( - MinFixedLenByteArrayStatsIterator, - min_bytes, - ParquetStatistics::FixedLenByteArray, - [u8] -); -make_stats_iterator!( - MaxFixedLenByteArrayStatsIterator, - max_bytes, - ParquetStatistics::FixedLenByteArray, - [u8] -); - -/// Special iterator adapter for extracting i128 values from from an iterator of -/// `ParquetStatistics` -/// -/// Handles checking if the statistics are present and valid with the correct type. -/// -/// Depending on the parquet file, the statistics for `Decimal128` can be stored as -/// `Int32`, `Int64` or `ByteArray` or `FixedSizeByteArray` :mindblown: -/// -/// This iterator handles all cases, extracting the values -/// and converting it to `stat_value_type`. -/// -/// Parameters: -/// * `$iterator_type` is the name of the iterator type (e.g. `MinBooleanStatsIterator`) -/// * `$func` is the function to call to get the value (e.g. `min` or `max`) -/// * `$bytes_func` is the function to call to get the value as bytes (e.g. `min_bytes` or `max_bytes`) -/// * `$stat_value_type` is the type of the statistics value (e.g. `i128`) -/// * `convert_func` is the function to convert the bytes to stats value (e.g. `from_bytes_to_i128`) -macro_rules! make_decimal_stats_iterator { - ($iterator_type:ident, $func:ident, $bytes_func:ident, $stat_value_type:ident, $convert_func: ident) => { - struct $iterator_type<'a, I> - where - I: Iterator>, - { - iter: I, - } - - impl<'a, I> $iterator_type<'a, I> - where - I: Iterator>, - { - fn new(iter: I) -> Self { - Self { iter } - } - } - - impl<'a, I> Iterator for $iterator_type<'a, I> - where - I: Iterator>, - { - type Item = Option<$stat_value_type>; - - fn next(&mut self) -> Option { - let next = self.iter.next(); - next.map(|x| { - x.and_then(|stats| { - if !stats.has_min_max_set() { - return None; - } - match stats { - ParquetStatistics::Int32(s) => { - Some($stat_value_type::from(*s.$func())) - } - ParquetStatistics::Int64(s) => { - Some($stat_value_type::from(*s.$func())) - } - ParquetStatistics::ByteArray(s) => { - Some($convert_func(s.$bytes_func())) - } - ParquetStatistics::FixedLenByteArray(s) => { - Some($convert_func(s.$bytes_func())) - } - _ => None, - } - }) - }) - } - - fn size_hint(&self) -> (usize, Option) { - self.iter.size_hint() - } - } - }; -} - -make_decimal_stats_iterator!( - MinDecimal128StatsIterator, - min, - min_bytes, - i128, - from_bytes_to_i128 -); -make_decimal_stats_iterator!( - MaxDecimal128StatsIterator, - max, - max_bytes, - i128, - from_bytes_to_i128 -); -make_decimal_stats_iterator!( - MinDecimal256StatsIterator, - min, - min_bytes, - i256, - from_bytes_to_i256 -); -make_decimal_stats_iterator!( - MaxDecimal256StatsIterator, - max, - max_bytes, - i256, - from_bytes_to_i256 -); - -/// Special macro to combine the statistics iterators for min and max using the [`mod@paste`] macro. -/// This is used to avoid repeating the same code for min and max statistics extractions -/// -/// Parameters: -/// stat_type_prefix: The prefix of the statistics iterator type (e.g. `Min` or `Max`) -/// data_type: The data type of the statistics (e.g. `DataType::Int32`) -/// iterator: The iterator of [`ParquetStatistics`] to extract the statistics from. -macro_rules! get_statistics { - ($stat_type_prefix: ident, $data_type: ident, $iterator: ident) => { - paste! { - match $data_type { - DataType::Boolean => Ok(Arc::new(BooleanArray::from_iter( - [<$stat_type_prefix BooleanStatsIterator>]::new($iterator).map(|x| x.copied()), - ))), - DataType::Int8 => Ok(Arc::new(Int8Array::from_iter( - [<$stat_type_prefix Int32StatsIterator>]::new($iterator).map(|x| { - x.and_then(|x| i8::try_from(*x).ok()) - }), - ))), - DataType::Int16 => Ok(Arc::new(Int16Array::from_iter( - [<$stat_type_prefix Int32StatsIterator>]::new($iterator).map(|x| { - x.and_then(|x| i16::try_from(*x).ok()) - }), - ))), - DataType::Int32 => Ok(Arc::new(Int32Array::from_iter( - [<$stat_type_prefix Int32StatsIterator>]::new($iterator).map(|x| x.copied()), - ))), - DataType::Int64 => Ok(Arc::new(Int64Array::from_iter( - [<$stat_type_prefix Int64StatsIterator>]::new($iterator).map(|x| x.copied()), - ))), - DataType::UInt8 => Ok(Arc::new(UInt8Array::from_iter( - [<$stat_type_prefix Int32StatsIterator>]::new($iterator).map(|x| { - x.and_then(|x| u8::try_from(*x).ok()) - }), - ))), - DataType::UInt16 => Ok(Arc::new(UInt16Array::from_iter( - [<$stat_type_prefix Int32StatsIterator>]::new($iterator).map(|x| { - x.and_then(|x| u16::try_from(*x).ok()) - }), - ))), - DataType::UInt32 => Ok(Arc::new(UInt32Array::from_iter( - [<$stat_type_prefix Int32StatsIterator>]::new($iterator).map(|x| x.map(|x| *x as u32)), - ))), - DataType::UInt64 => Ok(Arc::new(UInt64Array::from_iter( - [<$stat_type_prefix Int64StatsIterator>]::new($iterator).map(|x| x.map(|x| *x as u64)), - ))), - DataType::Float16 => Ok(Arc::new(Float16Array::from_iter( - [<$stat_type_prefix FixedLenByteArrayStatsIterator>]::new($iterator).map(|x| x.and_then(|x| { - from_bytes_to_f16(x) - })), - ))), - DataType::Float32 => Ok(Arc::new(Float32Array::from_iter( - [<$stat_type_prefix FloatStatsIterator>]::new($iterator).map(|x| x.copied()), - ))), - DataType::Float64 => Ok(Arc::new(Float64Array::from_iter( - [<$stat_type_prefix DoubleStatsIterator>]::new($iterator).map(|x| x.copied()), - ))), - DataType::Date32 => Ok(Arc::new(Date32Array::from_iter( - [<$stat_type_prefix Int32StatsIterator>]::new($iterator).map(|x| x.copied()), - ))), - DataType::Date64 => Ok(Arc::new(Date64Array::from_iter( - [<$stat_type_prefix Int32StatsIterator>]::new($iterator) - .map(|x| x.map(|x| i64::from(*x) * 24 * 60 * 60 * 1000)), - ))), - DataType::Timestamp(unit, timezone) =>{ - let iter = [<$stat_type_prefix Int64StatsIterator>]::new($iterator).map(|x| x.copied()); - Ok(match unit { - TimeUnit::Second => Arc::new(TimestampSecondArray::from_iter(iter).with_timezone_opt(timezone.clone())), - TimeUnit::Millisecond => Arc::new(TimestampMillisecondArray::from_iter(iter).with_timezone_opt(timezone.clone())), - TimeUnit::Microsecond => Arc::new(TimestampMicrosecondArray::from_iter(iter).with_timezone_opt(timezone.clone())), - TimeUnit::Nanosecond => Arc::new(TimestampNanosecondArray::from_iter(iter).with_timezone_opt(timezone.clone())), - }) - }, - DataType::Time32(unit) => { - Ok(match unit { - TimeUnit::Second => Arc::new(Time32SecondArray::from_iter( - [<$stat_type_prefix Int32StatsIterator>]::new($iterator).map(|x| x.copied()), - )), - TimeUnit::Millisecond => Arc::new(Time32MillisecondArray::from_iter( - [<$stat_type_prefix Int32StatsIterator>]::new($iterator).map(|x| x.copied()), - )), - _ => { - let len = $iterator.count(); - // don't know how to extract statistics, so return a null array - new_null_array($data_type, len) - } - }) - }, - DataType::Time64(unit) => { - Ok(match unit { - TimeUnit::Microsecond => Arc::new(Time64MicrosecondArray::from_iter( - [<$stat_type_prefix Int64StatsIterator>]::new($iterator).map(|x| x.copied()), - )), - TimeUnit::Nanosecond => Arc::new(Time64NanosecondArray::from_iter( - [<$stat_type_prefix Int64StatsIterator>]::new($iterator).map(|x| x.copied()), - )), - _ => { - let len = $iterator.count(); - // don't know how to extract statistics, so return a null array - new_null_array($data_type, len) - } - }) - }, - DataType::Binary => Ok(Arc::new(BinaryArray::from_iter( - [<$stat_type_prefix ByteArrayStatsIterator>]::new($iterator) - ))), - DataType::LargeBinary => Ok(Arc::new(LargeBinaryArray::from_iter( - [<$stat_type_prefix ByteArrayStatsIterator>]::new($iterator) - ))), - DataType::Utf8 => { - let iterator = [<$stat_type_prefix ByteArrayStatsIterator>]::new($iterator); - let mut builder = StringBuilder::new(); - for x in iterator { - let Some(x) = x else { - builder.append_null(); // no statistics value - continue; - }; - - let Ok(x) = std::str::from_utf8(x) else { - log::debug!("Utf8 statistics is a non-UTF8 value, ignoring it."); - builder.append_null(); - continue; - }; - - builder.append_value(x); - } - Ok(Arc::new(builder.finish())) - }, - DataType::LargeUtf8 => { - let iterator = [<$stat_type_prefix ByteArrayStatsIterator>]::new($iterator); - let mut builder = LargeStringBuilder::new(); - for x in iterator { - let Some(x) = x else { - builder.append_null(); // no statistics value - continue; - }; - - let Ok(x) = std::str::from_utf8(x) else { - log::debug!("Utf8 statistics is a non-UTF8 value, ignoring it."); - builder.append_null(); - continue; - }; - - builder.append_value(x); - } - Ok(Arc::new(builder.finish())) - }, - DataType::FixedSizeBinary(size) => { - let iterator = [<$stat_type_prefix FixedLenByteArrayStatsIterator>]::new($iterator); - let mut builder = FixedSizeBinaryBuilder::new(*size); - for x in iterator { - let Some(x) = x else { - builder.append_null(); // no statistics value - continue; - }; - - // ignore invalid values - if x.len().try_into() != Ok(*size){ - log::debug!( - "FixedSizeBinary({}) statistics is a binary of size {}, ignoring it.", - size, - x.len(), - ); - builder.append_null(); - continue; - } - - builder.append_value(x).expect("ensure to append successfully here, because size have been checked before"); - } - Ok(Arc::new(builder.finish())) - }, - DataType::Decimal128(precision, scale) => { - let arr = Decimal128Array::from_iter( - [<$stat_type_prefix Decimal128StatsIterator>]::new($iterator) - ).with_precision_and_scale(*precision, *scale)?; - Ok(Arc::new(arr)) - }, - DataType::Decimal256(precision, scale) => { - let arr = Decimal256Array::from_iter( - [<$stat_type_prefix Decimal256StatsIterator>]::new($iterator) - ).with_precision_and_scale(*precision, *scale)?; - Ok(Arc::new(arr)) - }, - DataType::Dictionary(_, value_type) => { - [<$stat_type_prefix:lower _ statistics>](value_type, $iterator) - } - - DataType::Map(_,_) | - DataType::Duration(_) | - DataType::Interval(_) | - DataType::Null | - DataType::BinaryView | - DataType::Utf8View | - DataType::List(_) | - DataType::ListView(_) | - DataType::FixedSizeList(_, _) | - DataType::LargeList(_) | - DataType::LargeListView(_) | - DataType::Struct(_) | - DataType::Union(_, _) | - DataType::RunEndEncoded(_, _) => { - let len = $iterator.count(); - // don't know how to extract statistics, so return a null array - Ok(new_null_array($data_type, len)) - } - }}} -} - -macro_rules! make_data_page_stats_iterator { - ($iterator_type: ident, $func: expr, $index_type: path, $stat_value_type: ty) => { - struct $iterator_type<'a, I> - where - I: Iterator, - { - iter: I, - } - - impl<'a, I> $iterator_type<'a, I> - where - I: Iterator, - { - fn new(iter: I) -> Self { - Self { iter } - } - } - - impl<'a, I> Iterator for $iterator_type<'a, I> - where - I: Iterator, - { - type Item = Vec>; - - fn next(&mut self) -> Option { - let next = self.iter.next(); - match next { - Some((len, index)) => match index { - $index_type(native_index) => Some( - native_index - .indexes - .iter() - .map(|x| $func(x)) - .collect::>(), - ), - // No matching `Index` found; - // thus no statistics that can be extracted. - // We return vec![None; len] to effectively - // create an arrow null-array with the length - // corresponding to the number of entries in - // `ParquetOffsetIndex` per row group per column. - _ => Some(vec![None; len]), - }, - _ => None, - } - } - - fn size_hint(&self) -> (usize, Option) { - self.iter.size_hint() - } - } - }; -} - -make_data_page_stats_iterator!( - MinBooleanDataPageStatsIterator, - |x: &PageIndex| { x.min }, - Index::BOOLEAN, - bool -); -make_data_page_stats_iterator!( - MaxBooleanDataPageStatsIterator, - |x: &PageIndex| { x.max }, - Index::BOOLEAN, - bool -); -make_data_page_stats_iterator!( - MinInt32DataPageStatsIterator, - |x: &PageIndex| { x.min }, - Index::INT32, - i32 -); -make_data_page_stats_iterator!( - MaxInt32DataPageStatsIterator, - |x: &PageIndex| { x.max }, - Index::INT32, - i32 -); -make_data_page_stats_iterator!( - MinInt64DataPageStatsIterator, - |x: &PageIndex| { x.min }, - Index::INT64, - i64 -); -make_data_page_stats_iterator!( - MaxInt64DataPageStatsIterator, - |x: &PageIndex| { x.max }, - Index::INT64, - i64 -); -make_data_page_stats_iterator!( - MinFloat16DataPageStatsIterator, - |x: &PageIndex| { x.min.clone() }, - Index::FIXED_LEN_BYTE_ARRAY, - FixedLenByteArray -); -make_data_page_stats_iterator!( - MaxFloat16DataPageStatsIterator, - |x: &PageIndex| { x.max.clone() }, - Index::FIXED_LEN_BYTE_ARRAY, - FixedLenByteArray -); -make_data_page_stats_iterator!( - MinFloat32DataPageStatsIterator, - |x: &PageIndex| { x.min }, - Index::FLOAT, - f32 -); -make_data_page_stats_iterator!( - MaxFloat32DataPageStatsIterator, - |x: &PageIndex| { x.max }, - Index::FLOAT, - f32 -); -make_data_page_stats_iterator!( - MinFloat64DataPageStatsIterator, - |x: &PageIndex| { x.min }, - Index::DOUBLE, - f64 -); -make_data_page_stats_iterator!( - MaxFloat64DataPageStatsIterator, - |x: &PageIndex| { x.max }, - Index::DOUBLE, - f64 -); -make_data_page_stats_iterator!( - MinByteArrayDataPageStatsIterator, - |x: &PageIndex| { x.min.clone() }, - Index::BYTE_ARRAY, - ByteArray -); -make_data_page_stats_iterator!( - MaxByteArrayDataPageStatsIterator, - |x: &PageIndex| { x.max.clone() }, - Index::BYTE_ARRAY, - ByteArray -); -make_data_page_stats_iterator!( - MaxFixedLenByteArrayDataPageStatsIterator, - |x: &PageIndex| { x.max.clone() }, - Index::FIXED_LEN_BYTE_ARRAY, - FixedLenByteArray -); - -make_data_page_stats_iterator!( - MinFixedLenByteArrayDataPageStatsIterator, - |x: &PageIndex| { x.min.clone() }, - Index::FIXED_LEN_BYTE_ARRAY, - FixedLenByteArray -); - -macro_rules! get_decimal_page_stats_iterator { - ($iterator_type: ident, $func: ident, $stat_value_type: ident, $convert_func: ident) => { - struct $iterator_type<'a, I> - where - I: Iterator, - { - iter: I, - } - - impl<'a, I> $iterator_type<'a, I> - where - I: Iterator, - { - fn new(iter: I) -> Self { - Self { iter } - } - } - - impl<'a, I> Iterator for $iterator_type<'a, I> - where - I: Iterator, - { - type Item = Vec>; - - fn next(&mut self) -> Option { - let next = self.iter.next(); - match next { - Some((len, index)) => match index { - Index::INT32(native_index) => Some( - native_index - .indexes - .iter() - .map(|x| { - x.$func.and_then(|x| Some($stat_value_type::from(x))) - }) - .collect::>(), - ), - Index::INT64(native_index) => Some( - native_index - .indexes - .iter() - .map(|x| { - x.$func.and_then(|x| Some($stat_value_type::from(x))) - }) - .collect::>(), - ), - Index::BYTE_ARRAY(native_index) => Some( - native_index - .indexes - .iter() - .map(|x| { - x.clone() - .$func - .and_then(|x| Some($convert_func(x.data()))) - }) - .collect::>(), - ), - Index::FIXED_LEN_BYTE_ARRAY(native_index) => Some( - native_index - .indexes - .iter() - .map(|x| { - x.clone() - .$func - .and_then(|x| Some($convert_func(x.data()))) - }) - .collect::>(), - ), - _ => Some(vec![None; len]), - }, - _ => None, - } - } - - fn size_hint(&self) -> (usize, Option) { - self.iter.size_hint() - } - } - }; -} - -get_decimal_page_stats_iterator!( - MinDecimal128DataPageStatsIterator, - min, - i128, - from_bytes_to_i128 -); - -get_decimal_page_stats_iterator!( - MaxDecimal128DataPageStatsIterator, - max, - i128, - from_bytes_to_i128 -); - -get_decimal_page_stats_iterator!( - MinDecimal256DataPageStatsIterator, - min, - i256, - from_bytes_to_i256 -); - -get_decimal_page_stats_iterator!( - MaxDecimal256DataPageStatsIterator, - max, - i256, - from_bytes_to_i256 -); - -macro_rules! get_data_page_statistics { - ($stat_type_prefix: ident, $data_type: ident, $iterator: ident) => { - paste! { - match $data_type { - Some(DataType::Boolean) => { - let iterator = [<$stat_type_prefix BooleanDataPageStatsIterator>]::new($iterator); - let mut builder = BooleanBuilder::new(); - for x in iterator { - for x in x.into_iter() { - let Some(x) = x else { - builder.append_null(); // no statistics value - continue; - }; - builder.append_value(x); - } - } - Ok(Arc::new(builder.finish())) - }, - Some(DataType::UInt8) => Ok(Arc::new( - UInt8Array::from_iter( - [<$stat_type_prefix Int32DataPageStatsIterator>]::new($iterator) - .map(|x| { - x.into_iter().map(|x| { - x.and_then(|x| u8::try_from(x).ok()) - }) - }) - .flatten() - ) - )), - Some(DataType::UInt16) => Ok(Arc::new( - UInt16Array::from_iter( - [<$stat_type_prefix Int32DataPageStatsIterator>]::new($iterator) - .map(|x| { - x.into_iter().map(|x| { - x.and_then(|x| u16::try_from(x).ok()) - }) - }) - .flatten() - ) - )), - Some(DataType::UInt32) => Ok(Arc::new( - UInt32Array::from_iter( - [<$stat_type_prefix Int32DataPageStatsIterator>]::new($iterator) - .map(|x| { - x.into_iter().map(|x| { - x.and_then(|x| Some(x as u32)) - }) - }) - .flatten() - ))), - Some(DataType::UInt64) => Ok(Arc::new( - UInt64Array::from_iter( - [<$stat_type_prefix Int64DataPageStatsIterator>]::new($iterator) - .map(|x| { - x.into_iter().map(|x| { - x.and_then(|x| Some(x as u64)) - }) - }) - .flatten() - ))), - Some(DataType::Int8) => Ok(Arc::new( - Int8Array::from_iter( - [<$stat_type_prefix Int32DataPageStatsIterator>]::new($iterator) - .map(|x| { - x.into_iter().map(|x| { - x.and_then(|x| i8::try_from(x).ok()) - }) - }) - .flatten() - ) - )), - Some(DataType::Int16) => Ok(Arc::new( - Int16Array::from_iter( - [<$stat_type_prefix Int32DataPageStatsIterator>]::new($iterator) - .map(|x| { - x.into_iter().map(|x| { - x.and_then(|x| i16::try_from(x).ok()) - }) - }) - .flatten() - ) - )), - Some(DataType::Int32) => Ok(Arc::new(Int32Array::from_iter([<$stat_type_prefix Int32DataPageStatsIterator>]::new($iterator).flatten()))), - Some(DataType::Int64) => Ok(Arc::new(Int64Array::from_iter([<$stat_type_prefix Int64DataPageStatsIterator>]::new($iterator).flatten()))), - Some(DataType::Float16) => Ok(Arc::new( - Float16Array::from_iter( - [<$stat_type_prefix Float16DataPageStatsIterator>]::new($iterator) - .map(|x| { - x.into_iter().map(|x| { - x.and_then(|x| from_bytes_to_f16(x.data())) - }) - }) - .flatten() - ) - )), - Some(DataType::Float32) => Ok(Arc::new(Float32Array::from_iter([<$stat_type_prefix Float32DataPageStatsIterator>]::new($iterator).flatten()))), - Some(DataType::Float64) => Ok(Arc::new(Float64Array::from_iter([<$stat_type_prefix Float64DataPageStatsIterator>]::new($iterator).flatten()))), - Some(DataType::Binary) => Ok(Arc::new(BinaryArray::from_iter([<$stat_type_prefix ByteArrayDataPageStatsIterator>]::new($iterator).flatten()))), - Some(DataType::LargeBinary) => Ok(Arc::new(LargeBinaryArray::from_iter([<$stat_type_prefix ByteArrayDataPageStatsIterator>]::new($iterator).flatten()))), - Some(DataType::Utf8) => { - let mut builder = StringBuilder::new(); - let iterator = [<$stat_type_prefix ByteArrayDataPageStatsIterator>]::new($iterator); - for x in iterator { - for x in x.into_iter() { - let Some(x) = x else { - builder.append_null(); // no statistics value - continue; - }; - - let Ok(x) = std::str::from_utf8(x.data()) else { - log::debug!("Utf8 statistics is a non-UTF8 value, ignoring it."); - builder.append_null(); - continue; - }; - - builder.append_value(x); - } - } - Ok(Arc::new(builder.finish())) - }, - Some(DataType::LargeUtf8) => { - let mut builder = LargeStringBuilder::new(); - let iterator = [<$stat_type_prefix ByteArrayDataPageStatsIterator>]::new($iterator); - for x in iterator { - for x in x.into_iter() { - let Some(x) = x else { - builder.append_null(); // no statistics value - continue; - }; - - let Ok(x) = std::str::from_utf8(x.data()) else { - log::debug!("LargeUtf8 statistics is a non-UTF8 value, ignoring it."); - builder.append_null(); - continue; - }; - - builder.append_value(x); - } - } - Ok(Arc::new(builder.finish())) - }, - Some(DataType::Dictionary(_, value_type)) => { - [<$stat_type_prefix:lower _ page_statistics>](Some(value_type), $iterator) - }, - Some(DataType::Timestamp(unit, timezone)) => { - let iter = [<$stat_type_prefix Int64DataPageStatsIterator>]::new($iterator).flatten(); - Ok(match unit { - TimeUnit::Second => Arc::new(TimestampSecondArray::from_iter(iter).with_timezone_opt(timezone.clone())), - TimeUnit::Millisecond => Arc::new(TimestampMillisecondArray::from_iter(iter).with_timezone_opt(timezone.clone())), - TimeUnit::Microsecond => Arc::new(TimestampMicrosecondArray::from_iter(iter).with_timezone_opt(timezone.clone())), - TimeUnit::Nanosecond => Arc::new(TimestampNanosecondArray::from_iter(iter).with_timezone_opt(timezone.clone())), - }) - }, - Some(DataType::Date32) => Ok(Arc::new(Date32Array::from_iter([<$stat_type_prefix Int32DataPageStatsIterator>]::new($iterator).flatten()))), - Some(DataType::Date64) => Ok( - Arc::new( - Date64Array::from_iter([<$stat_type_prefix Int32DataPageStatsIterator>]::new($iterator) - .map(|x| { - x.into_iter() - .map(|x| { - x.and_then(|x| i64::try_from(x).ok()) - }) - .map(|x| x.map(|x| x * 24 * 60 * 60 * 1000)) - }).flatten() - ) - ) - ), - Some(DataType::Decimal128(precision, scale)) => Ok(Arc::new( - Decimal128Array::from_iter([<$stat_type_prefix Decimal128DataPageStatsIterator>]::new($iterator).flatten()).with_precision_and_scale(*precision, *scale)?)), - Some(DataType::Decimal256(precision, scale)) => Ok(Arc::new( - Decimal256Array::from_iter([<$stat_type_prefix Decimal256DataPageStatsIterator>]::new($iterator).flatten()).with_precision_and_scale(*precision, *scale)?)), - Some(DataType::Time32(unit)) => { - Ok(match unit { - TimeUnit::Second => Arc::new(Time32SecondArray::from_iter( - [<$stat_type_prefix Int32DataPageStatsIterator>]::new($iterator).flatten(), - )), - TimeUnit::Millisecond => Arc::new(Time32MillisecondArray::from_iter( - [<$stat_type_prefix Int32DataPageStatsIterator>]::new($iterator).flatten(), - )), - _ => { - // don't know how to extract statistics, so return an empty array - new_empty_array(&DataType::Time32(unit.clone())) - } - }) - } - Some(DataType::Time64(unit)) => { - Ok(match unit { - TimeUnit::Microsecond => Arc::new(Time64MicrosecondArray::from_iter( - [<$stat_type_prefix Int64DataPageStatsIterator>]::new($iterator).flatten(), - )), - TimeUnit::Nanosecond => Arc::new(Time64NanosecondArray::from_iter( - [<$stat_type_prefix Int64DataPageStatsIterator>]::new($iterator).flatten(), - )), - _ => { - // don't know how to extract statistics, so return an empty array - new_empty_array(&DataType::Time64(unit.clone())) - } - }) - }, - Some(DataType::FixedSizeBinary(size)) => { - let mut builder = FixedSizeBinaryBuilder::new(*size); - let iterator = [<$stat_type_prefix FixedLenByteArrayDataPageStatsIterator>]::new($iterator); - for x in iterator { - for x in x.into_iter() { - let Some(x) = x else { - builder.append_null(); // no statistics value - continue; - }; - - if x.len() == *size as usize { - let _ = builder.append_value(x.data()); - } else { - log::debug!( - "FixedSizeBinary({}) statistics is a binary of size {}, ignoring it.", - size, - x.len(), - ); - builder.append_null(); - } - } - } - Ok(Arc::new(builder.finish())) - }, - _ => unimplemented!() - } - } - } -} - -/// Lookups up the parquet column by name -/// -/// Returns the parquet column index and the corresponding arrow field -pub(crate) fn parquet_column<'a>( - parquet_schema: &SchemaDescriptor, - arrow_schema: &'a Schema, - name: &str, -) -> Option<(usize, &'a FieldRef)> { - let (root_idx, field) = arrow_schema.fields.find(name)?; - if field.data_type().is_nested() { - // Nested fields are not supported and require non-trivial logic - // to correctly walk the parquet schema accounting for the - // logical type rules - - // - // For example a ListArray could correspond to anything from 1 to 3 levels - // in the parquet schema - return None; - } - - // This could be made more efficient (#TBD) - let parquet_idx = (0..parquet_schema.columns().len()) - .find(|x| parquet_schema.get_column_root_idx(*x) == root_idx)?; - Some((parquet_idx, field)) -} - -/// Extracts the min statistics from an iterator of [`ParquetStatistics`] to an -/// [`ArrayRef`] -/// -/// This is an internal helper -- see [`StatisticsConverter`] for public API -fn min_statistics<'a, I: Iterator>>( - data_type: &DataType, - iterator: I, -) -> Result { - get_statistics!(Min, data_type, iterator) -} - -/// Extracts the max statistics from an iterator of [`ParquetStatistics`] to an [`ArrayRef`] -/// -/// This is an internal helper -- see [`StatisticsConverter`] for public API -fn max_statistics<'a, I: Iterator>>( - data_type: &DataType, - iterator: I, -) -> Result { - get_statistics!(Max, data_type, iterator) -} - -/// Extracts the min statistics from an iterator -/// of parquet page [`Index`]'es to an [`ArrayRef`] -pub(crate) fn min_page_statistics<'a, I>( - data_type: Option<&DataType>, - iterator: I, -) -> Result -where - I: Iterator, -{ - get_data_page_statistics!(Min, data_type, iterator) -} - -/// Extracts the max statistics from an iterator -/// of parquet page [`Index`]'es to an [`ArrayRef`] -pub(crate) fn max_page_statistics<'a, I>( - data_type: Option<&DataType>, - iterator: I, -) -> Result -where - I: Iterator, -{ - get_data_page_statistics!(Max, data_type, iterator) -} - -/// Extracts the null count statistics from an iterator -/// of parquet page [`Index`]'es to an [`ArrayRef`] -/// -/// The returned Array is an [`UInt64Array`] -pub(crate) fn null_counts_page_statistics<'a, I>(iterator: I) -> Result -where - I: Iterator, -{ - let iter = iterator.flat_map(|(len, index)| match index { - Index::NONE => vec![None; len], - Index::BOOLEAN(native_index) => native_index - .indexes - .iter() - .map(|x| x.null_count.map(|x| x as u64)) - .collect::>(), - Index::INT32(native_index) => native_index - .indexes - .iter() - .map(|x| x.null_count.map(|x| x as u64)) - .collect::>(), - Index::INT64(native_index) => native_index - .indexes - .iter() - .map(|x| x.null_count.map(|x| x as u64)) - .collect::>(), - Index::FLOAT(native_index) => native_index - .indexes - .iter() - .map(|x| x.null_count.map(|x| x as u64)) - .collect::>(), - Index::DOUBLE(native_index) => native_index - .indexes - .iter() - .map(|x| x.null_count.map(|x| x as u64)) - .collect::>(), - Index::FIXED_LEN_BYTE_ARRAY(native_index) => native_index - .indexes - .iter() - .map(|x| x.null_count.map(|x| x as u64)) - .collect::>(), - Index::BYTE_ARRAY(native_index) => native_index - .indexes - .iter() - .map(|x| x.null_count.map(|x| x as u64)) - .collect::>(), - _ => unimplemented!(), - }); - - Ok(UInt64Array::from_iter(iter)) -} - -/// Extracts Parquet statistics as Arrow arrays -/// -/// This is used to convert Parquet statistics to Arrow arrays, with proper type -/// conversions. This information can be used for pruning parquet files or row -/// groups based on the statistics embedded in parquet files -/// -/// # Schemas -/// -/// The schema of the parquet file and the arrow schema are used to convert the -/// underlying statistics value (stored as a parquet value) into the -/// corresponding Arrow value. For example, Decimals are stored as binary in -/// parquet files. -/// -/// The parquet_schema and arrow_schema do not have to be identical (for -/// example, the columns may be in different orders and one or the other schemas -/// may have additional columns). The function [`parquet_column`] is used to -/// match the column in the parquet file to the column in the arrow schema. -#[derive(Debug)] -pub struct StatisticsConverter<'a> { - /// the index of the matched column in the parquet schema - parquet_index: Option, - /// The field (with data type) of the column in the arrow schema - arrow_field: &'a Field, -} - -impl<'a> StatisticsConverter<'a> { - /// Returns a [`UInt64Array`] with row counts for each row group - /// - /// # Return Value - /// - /// The returned array has no nulls, and has one value for each row group. - /// Each value is the number of rows in the row group. - /// - /// # Example - /// ```no_run - /// # use arrow::datatypes::Schema; - /// # use arrow_array::ArrayRef; - /// # use parquet::file::metadata::ParquetMetaData; - /// # use datafusion::datasource::physical_plan::parquet::StatisticsConverter; - /// # fn get_parquet_metadata() -> ParquetMetaData { unimplemented!() } - /// # fn get_arrow_schema() -> Schema { unimplemented!() } - /// // Given the metadata for a parquet file and the arrow schema - /// let metadata: ParquetMetaData = get_parquet_metadata(); - /// let arrow_schema: Schema = get_arrow_schema(); - /// let parquet_schema = metadata.file_metadata().schema_descr(); - /// // create a converter - /// let converter = StatisticsConverter::try_new("foo", &arrow_schema, parquet_schema) - /// .unwrap(); - /// // get the row counts for each row group - /// let row_counts = converter.row_group_row_counts(metadata - /// .row_groups() - /// .iter() - /// ); - /// ``` - pub fn row_group_row_counts(&self, metadatas: I) -> Result> - where - I: IntoIterator, - { - let Some(_) = self.parquet_index else { - return Ok(None); - }; - - let mut builder = UInt64Array::builder(10); - for metadata in metadatas.into_iter() { - let row_count = metadata.num_rows(); - let row_count: u64 = row_count.try_into().map_err(|e| { - internal_datafusion_err!( - "Parquet row count {row_count} too large to convert to u64: {e}" - ) - })?; - builder.append_value(row_count); - } - Ok(Some(builder.finish())) - } - - /// Create a new `StatisticsConverter` to extract statistics for a column - /// - /// Note if there is no corresponding column in the parquet file, the returned - /// arrays will be null. This can happen if the column is in the arrow - /// schema but not in the parquet schema due to schema evolution. - /// - /// See example on [`Self::row_group_mins`] for usage - /// - /// # Errors - /// - /// * If the column is not found in the arrow schema - pub fn try_new<'b>( - column_name: &'b str, - arrow_schema: &'a Schema, - parquet_schema: &'a SchemaDescriptor, - ) -> Result { - // ensure the requested column is in the arrow schema - let Some((_idx, arrow_field)) = arrow_schema.column_with_name(column_name) else { - return plan_err!( - "Column '{}' not found in schema for statistics conversion", - column_name - ); - }; - - // find the column in the parquet schema, if not, return a null array - let parquet_index = match parquet_column( - parquet_schema, - arrow_schema, - column_name, - ) { - Some((parquet_idx, matched_field)) => { - // sanity check that matching field matches the arrow field - if matched_field.as_ref() != arrow_field { - return internal_err!( - "Matched column '{:?}' does not match original matched column '{:?}'", - matched_field, - arrow_field - ); - } - Some(parquet_idx) - } - None => None, - }; - - Ok(Self { - parquet_index, - arrow_field, - }) - } - - /// Extract the minimum values from row group statistics in [`RowGroupMetaData`] - /// - /// # Return Value - /// - /// The returned array contains 1 value for each row group, in the same order as `metadatas` - /// - /// Each value is either - /// * the minimum value for the column - /// * a null value, if the statistics can not be extracted - /// - /// Note that a null value does NOT mean the min value was actually - /// `null` it means it the requested statistic is unknown - /// - /// # Errors - /// - /// Reasons for not being able to extract the statistics include: - /// * the column is not present in the parquet file - /// * statistics for the column are not present in the row group - /// * the stored statistic value can not be converted to the requested type - /// - /// # Example - /// ```no_run - /// # use arrow::datatypes::Schema; - /// # use arrow_array::ArrayRef; - /// # use parquet::file::metadata::ParquetMetaData; - /// # use datafusion::datasource::physical_plan::parquet::StatisticsConverter; - /// # fn get_parquet_metadata() -> ParquetMetaData { unimplemented!() } - /// # fn get_arrow_schema() -> Schema { unimplemented!() } - /// // Given the metadata for a parquet file and the arrow schema - /// let metadata: ParquetMetaData = get_parquet_metadata(); - /// let arrow_schema: Schema = get_arrow_schema(); - /// let parquet_schema = metadata.file_metadata().schema_descr(); - /// // create a converter - /// let converter = StatisticsConverter::try_new("foo", &arrow_schema, parquet_schema) - /// .unwrap(); - /// // get the minimum value for the column "foo" in the parquet file - /// let min_values: ArrayRef = converter - /// .row_group_mins(metadata.row_groups().iter()) - /// .unwrap(); - /// ``` - pub fn row_group_mins(&self, metadatas: I) -> Result - where - I: IntoIterator, - { - let data_type = self.arrow_field.data_type(); - - let Some(parquet_index) = self.parquet_index else { - return Ok(self.make_null_array(data_type, metadatas)); - }; - - let iter = metadatas - .into_iter() - .map(|x| x.column(parquet_index).statistics()); - min_statistics(data_type, iter) - } - - /// Extract the maximum values from row group statistics in [`RowGroupMetaData`] - /// - /// See docs on [`Self::row_group_mins`] for details - pub fn row_group_maxes(&self, metadatas: I) -> Result - where - I: IntoIterator, - { - let data_type = self.arrow_field.data_type(); - - let Some(parquet_index) = self.parquet_index else { - return Ok(self.make_null_array(data_type, metadatas)); - }; - - let iter = metadatas - .into_iter() - .map(|x| x.column(parquet_index).statistics()); - max_statistics(data_type, iter) - } - - /// Extract the null counts from row group statistics in [`RowGroupMetaData`] - /// - /// See docs on [`Self::row_group_mins`] for details - pub fn row_group_null_counts(&self, metadatas: I) -> Result - where - I: IntoIterator, - { - let Some(parquet_index) = self.parquet_index else { - let num_row_groups = metadatas.into_iter().count(); - return Ok(UInt64Array::from_iter( - std::iter::repeat(None).take(num_row_groups), - )); - }; - - let null_counts = metadatas - .into_iter() - .map(|x| x.column(parquet_index).statistics()) - .map(|s| s.map(|s| s.null_count())); - Ok(UInt64Array::from_iter(null_counts)) - } - - /// Extract the minimum values from Data Page statistics. - /// - /// In Parquet files, in addition to the Column Chunk level statistics - /// (stored for each column for each row group) there are also - /// optional statistics stored for each data page, as part of - /// the [`ParquetColumnIndex`]. - /// - /// Since a single Column Chunk is stored as one or more pages, - /// page level statistics can prune at a finer granularity. - /// - /// However since they are stored in a separate metadata - /// structure ([`Index`]) there is different code to extract them as - /// compared to arrow statistics. - /// - /// # Parameters: - /// - /// * `column_page_index`: The parquet column page indices, read from - /// `ParquetMetaData` column_index - /// - /// * `column_offset_index`: The parquet column offset indices, read from - /// `ParquetMetaData` offset_index - /// - /// * `row_group_indices`: The indices of the row groups, that are used to - /// extract the column page index and offset index on a per row group - /// per column basis. - /// - /// # Return Value - /// - /// The returned array contains 1 value for each `NativeIndex` - /// in the underlying `Index`es, in the same order as they appear - /// in `metadatas`. - /// - /// For example, if there are two `Index`es in `metadatas`: - /// 1. the first having `3` `PageIndex` entries - /// 2. the second having `2` `PageIndex` entries - /// - /// The returned array would have 5 rows. - /// - /// Each value is either: - /// * the minimum value for the page - /// * a null value, if the statistics can not be extracted - /// - /// Note that a null value does NOT mean the min value was actually - /// `null` it means it the requested statistic is unknown - /// - /// # Errors - /// - /// Reasons for not being able to extract the statistics include: - /// * the column is not present in the parquet file - /// * statistics for the pages are not present in the row group - /// * the stored statistic value can not be converted to the requested type - pub fn data_page_mins( - &self, - column_page_index: &ParquetColumnIndex, - column_offset_index: &ParquetOffsetIndex, - row_group_indices: I, - ) -> Result - where - I: IntoIterator, - { - let data_type = self.arrow_field.data_type(); - - let Some(parquet_index) = self.parquet_index else { - return Ok(self.make_null_array(data_type, row_group_indices)); - }; - - let iter = row_group_indices.into_iter().map(|rg_index| { - let column_page_index_per_row_group_per_column = - &column_page_index[*rg_index][parquet_index]; - let num_data_pages = &column_offset_index[*rg_index][parquet_index].len(); - - (*num_data_pages, column_page_index_per_row_group_per_column) - }); - - min_page_statistics(Some(data_type), iter) - } - - /// Extract the maximum values from Data Page statistics. - /// - /// See docs on [`Self::data_page_mins`] for details. - pub fn data_page_maxes( - &self, - column_page_index: &ParquetColumnIndex, - column_offset_index: &ParquetOffsetIndex, - row_group_indices: I, - ) -> Result - where - I: IntoIterator, - { - let data_type = self.arrow_field.data_type(); - - let Some(parquet_index) = self.parquet_index else { - return Ok(self.make_null_array(data_type, row_group_indices)); - }; - - let iter = row_group_indices.into_iter().map(|rg_index| { - let column_page_index_per_row_group_per_column = - &column_page_index[*rg_index][parquet_index]; - let num_data_pages = &column_offset_index[*rg_index][parquet_index].len(); - - (*num_data_pages, column_page_index_per_row_group_per_column) - }); - - max_page_statistics(Some(data_type), iter) - } - - /// Extract the null counts from Data Page statistics. - /// - /// The returned Array is an [`UInt64Array`] - /// - /// See docs on [`Self::data_page_mins`] for details. - pub fn data_page_null_counts( - &self, - column_page_index: &ParquetColumnIndex, - column_offset_index: &ParquetOffsetIndex, - row_group_indices: I, - ) -> Result - where - I: IntoIterator, - { - let Some(parquet_index) = self.parquet_index else { - let num_row_groups = row_group_indices.into_iter().count(); - return Ok(UInt64Array::from_iter( - std::iter::repeat(None).take(num_row_groups), - )); - }; - - let iter = row_group_indices.into_iter().map(|rg_index| { - let column_page_index_per_row_group_per_column = - &column_page_index[*rg_index][parquet_index]; - let num_data_pages = &column_offset_index[*rg_index][parquet_index].len(); - - (*num_data_pages, column_page_index_per_row_group_per_column) - }); - null_counts_page_statistics(iter) - } - - /// Returns an [`ArrayRef`] with row counts for each row group. - /// - /// This function iterates over the given row group indexes and computes - /// the row count for each page in the specified column. - /// - /// # Parameters: - /// - /// * `column_offset_index`: The parquet column offset indices, read from - /// `ParquetMetaData` offset_index - /// - /// * `row_group_metadatas`: The metadata slice of the row groups, read - /// from `ParquetMetaData` row_groups - /// - /// * `row_group_indices`: The indices of the row groups, that are used to - /// extract the column offset index on a per row group per column basis. - /// - /// See docs on [`Self::data_page_mins`] for details. - pub fn data_page_row_counts( - &self, - column_offset_index: &ParquetOffsetIndex, - row_group_metadatas: &'a [RowGroupMetaData], - row_group_indices: I, - ) -> Result> - where - I: IntoIterator, - { - let Some(parquet_index) = self.parquet_index else { - // no matching column found in parquet_index; - // thus we cannot extract page_locations in order to determine - // the row count on a per DataPage basis. - return Ok(None); - }; - - let mut row_count_total = Vec::new(); - for rg_idx in row_group_indices { - let page_locations = &column_offset_index[*rg_idx][parquet_index]; - - let row_count_per_page = page_locations.windows(2).map(|loc| { - Some(loc[1].first_row_index as u64 - loc[0].first_row_index as u64) - }); - - // append the last page row count - let num_rows_in_row_group = &row_group_metadatas[*rg_idx].num_rows(); - let row_count_per_page = row_count_per_page - .chain(std::iter::once(Some( - *num_rows_in_row_group as u64 - - page_locations.last().unwrap().first_row_index as u64, - ))) - .collect::>(); - - row_count_total.extend(row_count_per_page); - } - - Ok(Some(UInt64Array::from_iter(row_count_total))) - } - - /// Returns a null array of data_type with one element per row group - fn make_null_array(&self, data_type: &DataType, metadatas: I) -> ArrayRef - where - I: IntoIterator, - { - // column was in the arrow schema but not in the parquet schema, so return a null array - let num_row_groups = metadatas.into_iter().count(); - new_null_array(data_type, num_row_groups) - } -} - -#[cfg(test)] -mod test { - use super::*; - use arrow::compute::kernels::cast_utils::Parser; - use arrow::datatypes::{i256, Date32Type, Date64Type}; - use arrow_array::{ - new_empty_array, new_null_array, Array, BinaryArray, BooleanArray, Date32Array, - Date64Array, Decimal128Array, Decimal256Array, Float32Array, Float64Array, - Int16Array, Int32Array, Int64Array, Int8Array, LargeBinaryArray, RecordBatch, - StringArray, StructArray, TimestampNanosecondArray, - }; - use arrow_schema::{Field, SchemaRef}; - use bytes::Bytes; - use datafusion_common::test_util::parquet_test_data; - use parquet::arrow::arrow_reader::ArrowReaderBuilder; - use parquet::arrow::arrow_writer::ArrowWriter; - use parquet::file::metadata::{ParquetMetaData, RowGroupMetaData}; - use parquet::file::properties::{EnabledStatistics, WriterProperties}; - use std::path::PathBuf; - use std::sync::Arc; - - // TODO error cases (with parquet statistics that are mismatched in expected type) - - #[test] - fn roundtrip_empty() { - let empty_bool_array = new_empty_array(&DataType::Boolean); - Test { - input: empty_bool_array.clone(), - expected_min: empty_bool_array.clone(), - expected_max: empty_bool_array.clone(), - } - .run() - } - - #[test] - fn roundtrip_bool() { - Test { - input: bool_array([ - // row group 1 - Some(true), - None, - Some(true), - // row group 2 - Some(true), - Some(false), - None, - // row group 3 - None, - None, - None, - ]), - expected_min: bool_array([Some(true), Some(false), None]), - expected_max: bool_array([Some(true), Some(true), None]), - } - .run() - } - - #[test] - fn roundtrip_int32() { - Test { - input: i32_array([ - // row group 1 - Some(1), - None, - Some(3), - // row group 2 - Some(0), - Some(5), - None, - // row group 3 - None, - None, - None, - ]), - expected_min: i32_array([Some(1), Some(0), None]), - expected_max: i32_array([Some(3), Some(5), None]), - } - .run() - } - - #[test] - fn roundtrip_int64() { - Test { - input: i64_array([ - // row group 1 - Some(1), - None, - Some(3), - // row group 2 - Some(0), - Some(5), - None, - // row group 3 - None, - None, - None, - ]), - expected_min: i64_array([Some(1), Some(0), None]), - expected_max: i64_array(vec![Some(3), Some(5), None]), - } - .run() - } - - #[test] - fn roundtrip_f32() { - Test { - input: f32_array([ - // row group 1 - Some(1.0), - None, - Some(3.0), - // row group 2 - Some(-1.0), - Some(5.0), - None, - // row group 3 - None, - None, - None, - ]), - expected_min: f32_array([Some(1.0), Some(-1.0), None]), - expected_max: f32_array([Some(3.0), Some(5.0), None]), - } - .run() - } - - #[test] - fn roundtrip_f64() { - Test { - input: f64_array([ - // row group 1 - Some(1.0), - None, - Some(3.0), - // row group 2 - Some(-1.0), - Some(5.0), - None, - // row group 3 - None, - None, - None, - ]), - expected_min: f64_array([Some(1.0), Some(-1.0), None]), - expected_max: f64_array([Some(3.0), Some(5.0), None]), - } - .run() - } - - #[test] - fn roundtrip_timestamp() { - Test { - input: timestamp_seconds_array( - [ - // row group 1 - Some(1), - None, - Some(3), - // row group 2 - Some(9), - Some(5), - None, - // row group 3 - None, - None, - None, - ], - None, - ), - expected_min: timestamp_seconds_array([Some(1), Some(5), None], None), - expected_max: timestamp_seconds_array([Some(3), Some(9), None], None), - } - .run(); - - Test { - input: timestamp_milliseconds_array( - [ - // row group 1 - Some(1), - None, - Some(3), - // row group 2 - Some(9), - Some(5), - None, - // row group 3 - None, - None, - None, - ], - None, - ), - expected_min: timestamp_milliseconds_array([Some(1), Some(5), None], None), - expected_max: timestamp_milliseconds_array([Some(3), Some(9), None], None), - } - .run(); - - Test { - input: timestamp_microseconds_array( - [ - // row group 1 - Some(1), - None, - Some(3), - // row group 2 - Some(9), - Some(5), - None, - // row group 3 - None, - None, - None, - ], - None, - ), - expected_min: timestamp_microseconds_array([Some(1), Some(5), None], None), - expected_max: timestamp_microseconds_array([Some(3), Some(9), None], None), - } - .run(); - - Test { - input: timestamp_nanoseconds_array( - [ - // row group 1 - Some(1), - None, - Some(3), - // row group 2 - Some(9), - Some(5), - None, - // row group 3 - None, - None, - None, - ], - None, - ), - expected_min: timestamp_nanoseconds_array([Some(1), Some(5), None], None), - expected_max: timestamp_nanoseconds_array([Some(3), Some(9), None], None), - } - .run() - } - - #[test] - fn roundtrip_timestamp_timezoned() { - Test { - input: timestamp_seconds_array( - [ - // row group 1 - Some(1), - None, - Some(3), - // row group 2 - Some(9), - Some(5), - None, - // row group 3 - None, - None, - None, - ], - Some("UTC"), - ), - expected_min: timestamp_seconds_array([Some(1), Some(5), None], Some("UTC")), - expected_max: timestamp_seconds_array([Some(3), Some(9), None], Some("UTC")), - } - .run(); - - Test { - input: timestamp_milliseconds_array( - [ - // row group 1 - Some(1), - None, - Some(3), - // row group 2 - Some(9), - Some(5), - None, - // row group 3 - None, - None, - None, - ], - Some("UTC"), - ), - expected_min: timestamp_milliseconds_array( - [Some(1), Some(5), None], - Some("UTC"), - ), - expected_max: timestamp_milliseconds_array( - [Some(3), Some(9), None], - Some("UTC"), - ), - } - .run(); - - Test { - input: timestamp_microseconds_array( - [ - // row group 1 - Some(1), - None, - Some(3), - // row group 2 - Some(9), - Some(5), - None, - // row group 3 - None, - None, - None, - ], - Some("UTC"), - ), - expected_min: timestamp_microseconds_array( - [Some(1), Some(5), None], - Some("UTC"), - ), - expected_max: timestamp_microseconds_array( - [Some(3), Some(9), None], - Some("UTC"), - ), - } - .run(); - - Test { - input: timestamp_nanoseconds_array( - [ - // row group 1 - Some(1), - None, - Some(3), - // row group 2 - Some(9), - Some(5), - None, - // row group 3 - None, - None, - None, - ], - Some("UTC"), - ), - expected_min: timestamp_nanoseconds_array( - [Some(1), Some(5), None], - Some("UTC"), - ), - expected_max: timestamp_nanoseconds_array( - [Some(3), Some(9), None], - Some("UTC"), - ), - } - .run() - } - - #[test] - fn roundtrip_decimal() { - Test { - input: Arc::new( - Decimal128Array::from(vec![ - // row group 1 - Some(100), - None, - Some(22000), - // row group 2 - Some(500000), - Some(330000), - None, - // row group 3 - None, - None, - None, - ]) - .with_precision_and_scale(9, 2) - .unwrap(), - ), - expected_min: Arc::new( - Decimal128Array::from(vec![Some(100), Some(330000), None]) - .with_precision_and_scale(9, 2) - .unwrap(), - ), - expected_max: Arc::new( - Decimal128Array::from(vec![Some(22000), Some(500000), None]) - .with_precision_and_scale(9, 2) - .unwrap(), - ), - } - .run(); - - Test { - input: Arc::new( - Decimal256Array::from(vec![ - // row group 1 - Some(i256::from(100)), - None, - Some(i256::from(22000)), - // row group 2 - Some(i256::MAX), - Some(i256::MIN), - None, - // row group 3 - None, - None, - None, - ]) - .with_precision_and_scale(76, 76) - .unwrap(), - ), - expected_min: Arc::new( - Decimal256Array::from(vec![Some(i256::from(100)), Some(i256::MIN), None]) - .with_precision_and_scale(76, 76) - .unwrap(), - ), - expected_max: Arc::new( - Decimal256Array::from(vec![ - Some(i256::from(22000)), - Some(i256::MAX), - None, - ]) - .with_precision_and_scale(76, 76) - .unwrap(), - ), - } - .run() - } - - #[test] - fn roundtrip_utf8() { - Test { - input: utf8_array([ - // row group 1 - Some("A"), - None, - Some("Q"), - // row group 2 - Some("ZZ"), - Some("AA"), - None, - // row group 3 - None, - None, - None, - ]), - expected_min: utf8_array([Some("A"), Some("AA"), None]), - expected_max: utf8_array([Some("Q"), Some("ZZ"), None]), - } - .run() - } - - #[test] - fn roundtrip_struct() { - let mut test = Test { - input: struct_array(vec![ - // row group 1 - (Some(true), Some(1)), - (None, None), - (Some(true), Some(3)), - // row group 2 - (Some(true), Some(0)), - (Some(false), Some(5)), - (None, None), - // row group 3 - (None, None), - (None, None), - (None, None), - ]), - expected_min: struct_array(vec![ - (Some(true), Some(1)), - (Some(true), Some(0)), - (None, None), - ]), - - expected_max: struct_array(vec![ - (Some(true), Some(3)), - (Some(true), Some(0)), - (None, None), - ]), - }; - // Due to https://github.com/apache/datafusion/issues/8334, - // statistics for struct arrays are not supported - test.expected_min = - new_null_array(test.input.data_type(), test.expected_min.len()); - test.expected_max = - new_null_array(test.input.data_type(), test.expected_min.len()); - test.run() - } - - #[test] - fn roundtrip_binary() { - Test { - input: Arc::new(BinaryArray::from_opt_vec(vec![ - // row group 1 - Some(b"A"), - None, - Some(b"Q"), - // row group 2 - Some(b"ZZ"), - Some(b"AA"), - None, - // row group 3 - None, - None, - None, - ])), - expected_min: Arc::new(BinaryArray::from_opt_vec(vec![ - Some(b"A"), - Some(b"AA"), - None, - ])), - expected_max: Arc::new(BinaryArray::from_opt_vec(vec![ - Some(b"Q"), - Some(b"ZZ"), - None, - ])), - } - .run() - } - - #[test] - fn roundtrip_date32() { - Test { - input: date32_array(vec![ - // row group 1 - Some("2021-01-01"), - None, - Some("2021-01-03"), - // row group 2 - Some("2021-01-01"), - Some("2021-01-05"), - None, - // row group 3 - None, - None, - None, - ]), - expected_min: date32_array(vec![ - Some("2021-01-01"), - Some("2021-01-01"), - None, - ]), - expected_max: date32_array(vec![ - Some("2021-01-03"), - Some("2021-01-05"), - None, - ]), - } - .run() - } - - #[test] - fn roundtrip_date64() { - Test { - input: date64_array(vec![ - // row group 1 - Some("2021-01-01"), - None, - Some("2021-01-03"), - // row group 2 - Some("2021-01-01"), - Some("2021-01-05"), - None, - // row group 3 - None, - None, - None, - ]), - expected_min: date64_array(vec![ - Some("2021-01-01"), - Some("2021-01-01"), - None, - ]), - expected_max: date64_array(vec![ - Some("2021-01-03"), - Some("2021-01-05"), - None, - ]), - } - .run() - } - - #[test] - fn roundtrip_large_binary_array() { - let input: Vec> = vec![ - // row group 1 - Some(b"A"), - None, - Some(b"Q"), - // row group 2 - Some(b"ZZ"), - Some(b"AA"), - None, - // row group 3 - None, - None, - None, - ]; - - let expected_min: Vec> = vec![Some(b"A"), Some(b"AA"), None]; - let expected_max: Vec> = vec![Some(b"Q"), Some(b"ZZ"), None]; - - Test { - input: large_binary_array(input), - expected_min: large_binary_array(expected_min), - expected_max: large_binary_array(expected_max), - } - .run(); - } - - #[test] - fn struct_and_non_struct() { - // Ensures that statistics for an array that appears *after* a struct - // array are not wrong - let struct_col = struct_array(vec![ - // row group 1 - (Some(true), Some(1)), - (None, None), - (Some(true), Some(3)), - ]); - let int_col = i32_array([Some(100), Some(200), Some(300)]); - let expected_min = i32_array([Some(100)]); - let expected_max = i32_array(vec![Some(300)]); - - // use a name that shadows a name in the struct column - match struct_col.data_type() { - DataType::Struct(fields) => { - assert_eq!(fields.get(1).unwrap().name(), "int_col") - } - _ => panic!("unexpected data type for struct column"), - }; - - let input_batch = RecordBatch::try_from_iter([ - ("struct_col", struct_col), - ("int_col", int_col), - ]) - .unwrap(); - - let schema = input_batch.schema(); - - let metadata = parquet_metadata(schema.clone(), input_batch); - let parquet_schema = metadata.file_metadata().schema_descr(); - - // read the int_col statistics - let (idx, _) = parquet_column(parquet_schema, &schema, "int_col").unwrap(); - assert_eq!(idx, 2); - - let row_groups = metadata.row_groups(); - let converter = - StatisticsConverter::try_new("int_col", &schema, parquet_schema).unwrap(); - - let min = converter.row_group_mins(row_groups.iter()).unwrap(); - assert_eq!( - &min, - &expected_min, - "Min. Statistics\n\n{}\n\n", - DisplayStats(row_groups) - ); - - let max = converter.row_group_maxes(row_groups.iter()).unwrap(); - assert_eq!( - &max, - &expected_max, - "Max. Statistics\n\n{}\n\n", - DisplayStats(row_groups) - ); - } - - #[test] - fn nan_in_stats() { - // /parquet-testing/data/nan_in_stats.parquet - // row_groups: 1 - // "x": Double({min: Some(1.0), max: Some(NaN), distinct_count: None, null_count: 0, min_max_deprecated: false, min_max_backwards_compatible: false}) - - TestFile::new("nan_in_stats.parquet") - .with_column(ExpectedColumn { - name: "x", - expected_min: Arc::new(Float64Array::from(vec![Some(1.0)])), - expected_max: Arc::new(Float64Array::from(vec![Some(f64::NAN)])), - }) - .run(); - } - - #[test] - fn alltypes_plain() { - // /parquet-testing/data/datapage_v1-snappy-compressed-checksum.parquet - // row_groups: 1 - // (has no statistics) - TestFile::new("alltypes_plain.parquet") - // No column statistics should be read as NULL, but with the right type - .with_column(ExpectedColumn { - name: "id", - expected_min: i32_array([None]), - expected_max: i32_array([None]), - }) - .with_column(ExpectedColumn { - name: "bool_col", - expected_min: bool_array([None]), - expected_max: bool_array([None]), - }) - .run(); - } - - #[test] - fn alltypes_tiny_pages() { - // /parquet-testing/data/alltypes_tiny_pages.parquet - // row_groups: 1 - // "id": Int32({min: Some(0), max: Some(7299), distinct_count: None, null_count: 0, min_max_deprecated: false, min_max_backwards_compatible: false}) - // "bool_col": Boolean({min: Some(false), max: Some(true), distinct_count: None, null_count: 0, min_max_deprecated: false, min_max_backwards_compatible: false}) - // "tinyint_col": Int32({min: Some(0), max: Some(9), distinct_count: None, null_count: 0, min_max_deprecated: false, min_max_backwards_compatible: false}) - // "smallint_col": Int32({min: Some(0), max: Some(9), distinct_count: None, null_count: 0, min_max_deprecated: false, min_max_backwards_compatible: false}) - // "int_col": Int32({min: Some(0), max: Some(9), distinct_count: None, null_count: 0, min_max_deprecated: false, min_max_backwards_compatible: false}) - // "bigint_col": Int64({min: Some(0), max: Some(90), distinct_count: None, null_count: 0, min_max_deprecated: false, min_max_backwards_compatible: false}) - // "float_col": Float({min: Some(0.0), max: Some(9.9), distinct_count: None, null_count: 0, min_max_deprecated: false, min_max_backwards_compatible: false}) - // "double_col": Double({min: Some(0.0), max: Some(90.89999999999999), distinct_count: None, null_count: 0, min_max_deprecated: false, min_max_backwards_compatible: false}) - // "date_string_col": ByteArray({min: Some(ByteArray { data: "01/01/09" }), max: Some(ByteArray { data: "12/31/10" }), distinct_count: None, null_count: 0, min_max_deprecated: false, min_max_backwards_compatible: false}) - // "string_col": ByteArray({min: Some(ByteArray { data: "0" }), max: Some(ByteArray { data: "9" }), distinct_count: None, null_count: 0, min_max_deprecated: false, min_max_backwards_compatible: false}) - // "timestamp_col": Int96({min: None, max: None, distinct_count: None, null_count: 0, min_max_deprecated: true, min_max_backwards_compatible: true}) - // "year": Int32({min: Some(2009), max: Some(2010), distinct_count: None, null_count: 0, min_max_deprecated: false, min_max_backwards_compatible: false}) - // "month": Int32({min: Some(1), max: Some(12), distinct_count: None, null_count: 0, min_max_deprecated: false, min_max_backwards_compatible: false}) - TestFile::new("alltypes_tiny_pages.parquet") - .with_column(ExpectedColumn { - name: "id", - expected_min: i32_array([Some(0)]), - expected_max: i32_array([Some(7299)]), - }) - .with_column(ExpectedColumn { - name: "bool_col", - expected_min: bool_array([Some(false)]), - expected_max: bool_array([Some(true)]), - }) - .with_column(ExpectedColumn { - name: "tinyint_col", - expected_min: i8_array([Some(0)]), - expected_max: i8_array([Some(9)]), - }) - .with_column(ExpectedColumn { - name: "smallint_col", - expected_min: i16_array([Some(0)]), - expected_max: i16_array([Some(9)]), - }) - .with_column(ExpectedColumn { - name: "int_col", - expected_min: i32_array([Some(0)]), - expected_max: i32_array([Some(9)]), - }) - .with_column(ExpectedColumn { - name: "bigint_col", - expected_min: i64_array([Some(0)]), - expected_max: i64_array([Some(90)]), - }) - .with_column(ExpectedColumn { - name: "float_col", - expected_min: f32_array([Some(0.0)]), - expected_max: f32_array([Some(9.9)]), - }) - .with_column(ExpectedColumn { - name: "double_col", - expected_min: f64_array([Some(0.0)]), - expected_max: f64_array([Some(90.89999999999999)]), - }) - .with_column(ExpectedColumn { - name: "date_string_col", - expected_min: utf8_array([Some("01/01/09")]), - expected_max: utf8_array([Some("12/31/10")]), - }) - .with_column(ExpectedColumn { - name: "string_col", - expected_min: utf8_array([Some("0")]), - expected_max: utf8_array([Some("9")]), - }) - // File has no min/max for timestamp_col - .with_column(ExpectedColumn { - name: "timestamp_col", - expected_min: timestamp_nanoseconds_array([None], None), - expected_max: timestamp_nanoseconds_array([None], None), - }) - .with_column(ExpectedColumn { - name: "year", - expected_min: i32_array([Some(2009)]), - expected_max: i32_array([Some(2010)]), - }) - .with_column(ExpectedColumn { - name: "month", - expected_min: i32_array([Some(1)]), - expected_max: i32_array([Some(12)]), - }) - .run(); - } - - #[test] - fn fixed_length_decimal_legacy() { - // /parquet-testing/data/fixed_length_decimal_legacy.parquet - // row_groups: 1 - // "value": FixedLenByteArray({min: Some(FixedLenByteArray(ByteArray { data: Some(ByteBufferPtr { data: b"\0\0\0\0\0\xc8" }) })), max: Some(FixedLenByteArray(ByteArray { data: "\0\0\0\0\t`" })), distinct_count: None, null_count: 0, min_max_deprecated: true, min_max_backwards_compatible: true}) - - TestFile::new("fixed_length_decimal_legacy.parquet") - .with_column(ExpectedColumn { - name: "value", - expected_min: Arc::new( - Decimal128Array::from(vec![Some(200)]) - .with_precision_and_scale(13, 2) - .unwrap(), - ), - expected_max: Arc::new( - Decimal128Array::from(vec![Some(2400)]) - .with_precision_and_scale(13, 2) - .unwrap(), - ), - }) - .run(); - } - - const ROWS_PER_ROW_GROUP: usize = 3; - - /// Writes the input batch into a parquet file, with every every three rows as - /// their own row group, and compares the min/maxes to the expected values - struct Test { - input: ArrayRef, - expected_min: ArrayRef, - expected_max: ArrayRef, - } - - impl Test { - fn run(self) { - let Self { - input, - expected_min, - expected_max, - } = self; - - let input_batch = RecordBatch::try_from_iter([("c1", input)]).unwrap(); - - let schema = input_batch.schema(); - - let metadata = parquet_metadata(schema.clone(), input_batch); - let parquet_schema = metadata.file_metadata().schema_descr(); - - let row_groups = metadata.row_groups(); - - for field in schema.fields() { - if field.data_type().is_nested() { - let lookup = parquet_column(parquet_schema, &schema, field.name()); - assert_eq!(lookup, None); - continue; - } - - let converter = - StatisticsConverter::try_new(field.name(), &schema, parquet_schema) - .unwrap(); - - assert_eq!(converter.arrow_field, field.as_ref()); - - let mins = converter.row_group_mins(row_groups.iter()).unwrap(); - assert_eq!( - &mins, - &expected_min, - "Min. Statistics\n\n{}\n\n", - DisplayStats(row_groups) - ); - - let maxes = converter.row_group_maxes(row_groups.iter()).unwrap(); - assert_eq!( - &maxes, - &expected_max, - "Max. Statistics\n\n{}\n\n", - DisplayStats(row_groups) - ); - } - } - } - - /// Write the specified batches out as parquet and return the metadata - fn parquet_metadata(schema: SchemaRef, batch: RecordBatch) -> Arc { - let props = WriterProperties::builder() - .set_statistics_enabled(EnabledStatistics::Chunk) - .set_max_row_group_size(ROWS_PER_ROW_GROUP) - .build(); - - let mut buffer = Vec::new(); - let mut writer = ArrowWriter::try_new(&mut buffer, schema, Some(props)).unwrap(); - writer.write(&batch).unwrap(); - writer.close().unwrap(); - - let reader = ArrowReaderBuilder::try_new(Bytes::from(buffer)).unwrap(); - reader.metadata().clone() - } - - /// Formats the statistics nicely for display - struct DisplayStats<'a>(&'a [RowGroupMetaData]); - impl<'a> std::fmt::Display for DisplayStats<'a> { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - let row_groups = self.0; - writeln!(f, " row_groups: {}", row_groups.len())?; - for rg in row_groups { - for col in rg.columns() { - if let Some(statistics) = col.statistics() { - writeln!(f, " {}: {:?}", col.column_path(), statistics)?; - } - } - } - Ok(()) - } - } - - struct ExpectedColumn { - name: &'static str, - expected_min: ArrayRef, - expected_max: ArrayRef, - } - - /// Reads statistics out of the specified, and compares them to the expected values - struct TestFile { - file_name: &'static str, - expected_columns: Vec, - } - - impl TestFile { - fn new(file_name: &'static str) -> Self { - Self { - file_name, - expected_columns: Vec::new(), - } - } - - fn with_column(mut self, column: ExpectedColumn) -> Self { - self.expected_columns.push(column); - self - } - - /// Reads the specified parquet file and validates that the expected min/max - /// values for the specified columns are as expected. - fn run(self) { - let path = PathBuf::from(parquet_test_data()).join(self.file_name); - let file = std::fs::File::open(path).unwrap(); - let reader = ArrowReaderBuilder::try_new(file).unwrap(); - let arrow_schema = reader.schema(); - let metadata = reader.metadata(); - let row_groups = metadata.row_groups(); - let parquet_schema = metadata.file_metadata().schema_descr(); - - for expected_column in self.expected_columns { - let ExpectedColumn { - name, - expected_min, - expected_max, - } = expected_column; - - let converter = - StatisticsConverter::try_new(name, arrow_schema, parquet_schema) - .unwrap(); - let actual_min = converter.row_group_mins(row_groups.iter()).unwrap(); - assert_eq!(&expected_min, &actual_min, "column {name}"); - - let actual_max = converter.row_group_maxes(row_groups.iter()).unwrap(); - assert_eq!(&expected_max, &actual_max, "column {name}"); - } - } - } - - fn bool_array(input: impl IntoIterator>) -> ArrayRef { - let array: BooleanArray = input.into_iter().collect(); - Arc::new(array) - } - - fn i8_array(input: impl IntoIterator>) -> ArrayRef { - let array: Int8Array = input.into_iter().collect(); - Arc::new(array) - } - - fn i16_array(input: impl IntoIterator>) -> ArrayRef { - let array: Int16Array = input.into_iter().collect(); - Arc::new(array) - } - - fn i32_array(input: impl IntoIterator>) -> ArrayRef { - let array: Int32Array = input.into_iter().collect(); - Arc::new(array) - } - - fn i64_array(input: impl IntoIterator>) -> ArrayRef { - let array: Int64Array = input.into_iter().collect(); - Arc::new(array) - } - - fn f32_array(input: impl IntoIterator>) -> ArrayRef { - let array: Float32Array = input.into_iter().collect(); - Arc::new(array) - } - - fn f64_array(input: impl IntoIterator>) -> ArrayRef { - let array: Float64Array = input.into_iter().collect(); - Arc::new(array) - } - - fn timestamp_seconds_array( - input: impl IntoIterator>, - timzezone: Option<&str>, - ) -> ArrayRef { - let array: TimestampSecondArray = input.into_iter().collect(); - match timzezone { - Some(tz) => Arc::new(array.with_timezone(tz)), - None => Arc::new(array), - } - } - - fn timestamp_milliseconds_array( - input: impl IntoIterator>, - timzezone: Option<&str>, - ) -> ArrayRef { - let array: TimestampMillisecondArray = input.into_iter().collect(); - match timzezone { - Some(tz) => Arc::new(array.with_timezone(tz)), - None => Arc::new(array), - } - } - - fn timestamp_microseconds_array( - input: impl IntoIterator>, - timzezone: Option<&str>, - ) -> ArrayRef { - let array: TimestampMicrosecondArray = input.into_iter().collect(); - match timzezone { - Some(tz) => Arc::new(array.with_timezone(tz)), - None => Arc::new(array), - } - } - - fn timestamp_nanoseconds_array( - input: impl IntoIterator>, - timzezone: Option<&str>, - ) -> ArrayRef { - let array: TimestampNanosecondArray = input.into_iter().collect(); - match timzezone { - Some(tz) => Arc::new(array.with_timezone(tz)), - None => Arc::new(array), - } - } - - fn utf8_array<'a>(input: impl IntoIterator>) -> ArrayRef { - let array: StringArray = input - .into_iter() - .map(|s| s.map(|s| s.to_string())) - .collect(); - Arc::new(array) - } - - // returns a struct array with columns "bool_col" and "int_col" with the specified values - fn struct_array(input: Vec<(Option, Option)>) -> ArrayRef { - let boolean: BooleanArray = input.iter().map(|(b, _i)| b).collect(); - let int: Int32Array = input.iter().map(|(_b, i)| i).collect(); - - let nullable = true; - let struct_array = StructArray::from(vec![ - ( - Arc::new(Field::new("bool_col", DataType::Boolean, nullable)), - Arc::new(boolean) as ArrayRef, - ), - ( - Arc::new(Field::new("int_col", DataType::Int32, nullable)), - Arc::new(int) as ArrayRef, - ), - ]); - Arc::new(struct_array) - } - - fn date32_array<'a>(input: impl IntoIterator>) -> ArrayRef { - let array = Date32Array::from( - input - .into_iter() - .map(|s| Date32Type::parse(s.unwrap_or_default())) - .collect::>(), - ); - Arc::new(array) - } - - fn date64_array<'a>(input: impl IntoIterator>) -> ArrayRef { - let array = Date64Array::from( - input - .into_iter() - .map(|s| Date64Type::parse(s.unwrap_or_default())) - .collect::>(), - ); - Arc::new(array) - } - - fn large_binary_array<'a>( - input: impl IntoIterator>, - ) -> ArrayRef { - let array = - LargeBinaryArray::from(input.into_iter().collect::>>()); - - Arc::new(array) - } -} diff --git a/datafusion/core/tests/parquet/arrow_statistics.rs b/datafusion/core/tests/parquet/arrow_statistics.rs deleted file mode 100644 index 2b4ba0b17133..000000000000 --- a/datafusion/core/tests/parquet/arrow_statistics.rs +++ /dev/null @@ -1,2178 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -//! This file contains an end to end test of extracting statistics from parquet files. -//! It writes data into a parquet file, reads statistics and verifies they are correct - -use std::default::Default; -use std::fs::File; -use std::sync::Arc; - -use crate::parquet::{struct_array, Scenario}; -use arrow::compute::kernels::cast_utils::Parser; -use arrow::datatypes::{ - i256, Date32Type, Date64Type, TimestampMicrosecondType, TimestampMillisecondType, - TimestampNanosecondType, TimestampSecondType, -}; -use arrow_array::{ - make_array, new_null_array, Array, ArrayRef, BinaryArray, BooleanArray, Date32Array, - Date64Array, Decimal128Array, Decimal256Array, FixedSizeBinaryArray, Float16Array, - Float32Array, Float64Array, Int16Array, Int32Array, Int64Array, Int8Array, - LargeBinaryArray, LargeStringArray, RecordBatch, StringArray, Time32MillisecondArray, - Time32SecondArray, Time64MicrosecondArray, Time64NanosecondArray, - TimestampMicrosecondArray, TimestampMillisecondArray, TimestampNanosecondArray, - TimestampSecondArray, UInt16Array, UInt32Array, UInt64Array, UInt8Array, -}; -use arrow_schema::{DataType, Field, Schema, TimeUnit}; -use datafusion::datasource::physical_plan::parquet::StatisticsConverter; -use half::f16; -use parquet::arrow::arrow_reader::{ - ArrowReaderBuilder, ArrowReaderOptions, ParquetRecordBatchReaderBuilder, -}; -use parquet::arrow::ArrowWriter; -use parquet::file::properties::{EnabledStatistics, WriterProperties}; - -use super::make_test_file_rg; - -#[derive(Debug, Default, Clone)] -struct Int64Case { - /// Number of nulls in the column - null_values: usize, - /// Non null values in the range `[no_null_values_start, - /// no_null_values_end]`, one value for each row - no_null_values_start: i64, - no_null_values_end: i64, - /// Number of rows per row group - row_per_group: usize, - /// if specified, overrides default statistics settings - enable_stats: Option, - /// If specified, the number of values in each page - data_page_row_count_limit: Option, -} - -impl Int64Case { - /// Return a record batch with i64 with Null values - /// The first no_null_values_end - no_null_values_start values - /// are non-null with the specified range, the rest are null - fn make_int64_batches_with_null(&self) -> RecordBatch { - let schema = - Arc::new(Schema::new(vec![Field::new("i64", DataType::Int64, true)])); - - let v64: Vec = - (self.no_null_values_start as _..self.no_null_values_end as _).collect(); - - RecordBatch::try_new( - schema, - vec![make_array( - Int64Array::from_iter( - v64.into_iter() - .map(Some) - .chain(std::iter::repeat(None).take(self.null_values)), - ) - .to_data(), - )], - ) - .unwrap() - } - - // Create a parquet file with the specified settings - pub fn build(&self) -> ParquetRecordBatchReaderBuilder { - let batches = vec![self.make_int64_batches_with_null()]; - build_parquet_file( - self.row_per_group, - self.enable_stats, - self.data_page_row_count_limit, - batches, - ) - } -} - -fn build_parquet_file( - row_per_group: usize, - enable_stats: Option, - data_page_row_count_limit: Option, - batches: Vec, -) -> ParquetRecordBatchReaderBuilder { - let mut output_file = tempfile::Builder::new() - .prefix("parquert_statistics_test") - .suffix(".parquet") - .tempfile() - .expect("tempfile creation"); - - let mut builder = WriterProperties::builder().set_max_row_group_size(row_per_group); - if let Some(enable_stats) = enable_stats { - builder = builder.set_statistics_enabled(enable_stats); - } - if let Some(data_page_row_count_limit) = data_page_row_count_limit { - builder = builder.set_data_page_row_count_limit(data_page_row_count_limit); - } - let props = builder.build(); - - let schema = batches[0].schema(); - - let mut writer = ArrowWriter::try_new(&mut output_file, schema, Some(props)).unwrap(); - - // if we have a datapage limit send the batches in one at a time to give - // the writer a chance to be split into multiple pages - if data_page_row_count_limit.is_some() { - for batch in &batches { - for i in 0..batch.num_rows() { - writer.write(&batch.slice(i, 1)).expect("writing batch"); - } - } - } else { - for batch in &batches { - writer.write(batch).expect("writing batch"); - } - } - - let _file_meta = writer.close().unwrap(); - - let file = output_file.reopen().unwrap(); - let options = ArrowReaderOptions::new().with_page_index(true); - ArrowReaderBuilder::try_new_with_options(file, options).unwrap() -} - -/// Defines what data to create in a parquet file -#[derive(Debug, Clone, Copy)] -struct TestReader { - /// What data to create in the parquet file - scenario: Scenario, - /// Number of rows per row group - row_per_group: usize, -} - -impl TestReader { - /// Create a parquet file with the specified data, and return a - /// ParquetRecordBatchReaderBuilder opened to that file. - async fn build(self) -> ParquetRecordBatchReaderBuilder { - let TestReader { - scenario, - row_per_group, - } = self; - let file = make_test_file_rg(scenario, row_per_group).await; - - // open the file & get the reader - let file = file.reopen().unwrap(); - let options = ArrowReaderOptions::new().with_page_index(true); - ArrowReaderBuilder::try_new_with_options(file, options).unwrap() - } -} - -/// Which statistics should we check? -#[derive(Clone, Debug, Copy)] -enum Check { - /// Extract and check row group statistics - RowGroup, - /// Extract and check data page statistics - DataPage, - /// Extract and check both row group and data page statistics. - /// - /// Note if a row group contains a single data page, - /// the statistics for row groups and data pages are the same. - Both, -} - -impl Check { - fn row_group(&self) -> bool { - match self { - Self::RowGroup | Self::Both => true, - Self::DataPage => false, - } - } - - fn data_page(&self) -> bool { - match self { - Self::DataPage | Self::Both => true, - Self::RowGroup => false, - } - } -} - -/// Defines a test case for statistics extraction -struct Test<'a> { - /// The parquet file reader - reader: &'a ParquetRecordBatchReaderBuilder, - expected_min: ArrayRef, - expected_max: ArrayRef, - expected_null_counts: UInt64Array, - expected_row_counts: Option, - /// Which column to extract statistics from - column_name: &'static str, - /// What statistics should be checked? - check: Check, -} - -impl<'a> Test<'a> { - fn run(self) { - let converter = StatisticsConverter::try_new( - self.column_name, - self.reader.schema(), - self.reader.parquet_schema(), - ) - .unwrap(); - - self.run_checks(converter); - } - - fn run_with_schema(self, schema: &Schema) { - let converter = StatisticsConverter::try_new( - self.column_name, - schema, - self.reader.parquet_schema(), - ) - .unwrap(); - - self.run_checks(converter); - } - - fn run_checks(self, converter: StatisticsConverter) { - let Self { - reader, - expected_min, - expected_max, - expected_null_counts, - expected_row_counts, - column_name, - check, - } = self; - - let row_groups = reader.metadata().row_groups(); - - if check.data_page() { - let column_page_index = reader - .metadata() - .column_index() - .expect("File should have column page indices"); - - let column_offset_index = reader - .metadata() - .offset_index() - .expect("File should have column offset indices"); - - let row_group_indices: Vec<_> = (0..row_groups.len()).collect(); - - let min = converter - .data_page_mins( - column_page_index, - column_offset_index, - &row_group_indices, - ) - .unwrap(); - assert_eq!( - &min, &expected_min, - "{column_name}: Mismatch with expected data page minimums" - ); - - let max = converter - .data_page_maxes( - column_page_index, - column_offset_index, - &row_group_indices, - ) - .unwrap(); - assert_eq!( - &max, &expected_max, - "{column_name}: Mismatch with expected data page maximum" - ); - - let null_counts = converter - .data_page_null_counts( - column_page_index, - column_offset_index, - &row_group_indices, - ) - .unwrap(); - - assert_eq!( - &null_counts, &expected_null_counts, - "{column_name}: Mismatch with expected data page null counts. \ - Actual: {null_counts:?}. Expected: {expected_null_counts:?}" - ); - - let row_counts = converter - .data_page_row_counts(column_offset_index, row_groups, &row_group_indices) - .unwrap(); - assert_eq!( - row_counts, expected_row_counts, - "{column_name}: Mismatch with expected row counts. \ - Actual: {row_counts:?}. Expected: {expected_row_counts:?}" - ); - } - - if check.row_group() { - let min = converter.row_group_mins(row_groups).unwrap(); - assert_eq!( - &min, &expected_min, - "{column_name}: Mismatch with expected minimums" - ); - - let max = converter.row_group_maxes(row_groups).unwrap(); - assert_eq!( - &max, &expected_max, - "{column_name}: Mismatch with expected maximum" - ); - - let null_counts = converter.row_group_null_counts(row_groups).unwrap(); - assert_eq!( - &null_counts, &expected_null_counts, - "{column_name}: Mismatch with expected null counts. \ - Actual: {null_counts:?}. Expected: {expected_null_counts:?}" - ); - - let row_counts = converter - .row_group_row_counts(reader.metadata().row_groups().iter()) - .unwrap(); - assert_eq!( - row_counts, expected_row_counts, - "{column_name}: Mismatch with expected row counts. \ - Actual: {row_counts:?}. Expected: {expected_row_counts:?}" - ); - } - } - - /// Run the test and expect a column not found error - fn run_col_not_found(self) { - let Self { - reader, - expected_min: _, - expected_max: _, - expected_null_counts: _, - expected_row_counts: _, - column_name, - .. - } = self; - - let converter = StatisticsConverter::try_new( - column_name, - reader.schema(), - reader.parquet_schema(), - ); - - assert!(converter.is_err()); - } -} - -// TESTS -// -// Remaining cases -// f64::NAN -// - Using truncated statistics ("exact min value" and "exact max value" https://docs.rs/parquet/latest/parquet/file/statistics/enum.Statistics.html#method.max_is_exact) - -#[tokio::test] -async fn test_one_row_group_without_null() { - let reader = Int64Case { - null_values: 0, - no_null_values_start: 4, - no_null_values_end: 7, - row_per_group: 20, - ..Default::default() - } - .build(); - - Test { - reader: &reader, - // min is 4 - expected_min: Arc::new(Int64Array::from(vec![4])), - // max is 6 - expected_max: Arc::new(Int64Array::from(vec![6])), - // no nulls - expected_null_counts: UInt64Array::from(vec![0]), - // 3 rows - expected_row_counts: Some(UInt64Array::from(vec![3])), - column_name: "i64", - check: Check::Both, - } - .run() -} - -#[tokio::test] -async fn test_one_row_group_with_null_and_negative() { - let reader = Int64Case { - null_values: 2, - no_null_values_start: -1, - no_null_values_end: 5, - row_per_group: 20, - ..Default::default() - } - .build(); - - Test { - reader: &reader, - // min is -1 - expected_min: Arc::new(Int64Array::from(vec![-1])), - // max is 4 - expected_max: Arc::new(Int64Array::from(vec![4])), - // 2 nulls - expected_null_counts: UInt64Array::from(vec![2]), - // 8 rows - expected_row_counts: Some(UInt64Array::from(vec![8])), - column_name: "i64", - check: Check::Both, - } - .run() -} - -#[tokio::test] -async fn test_two_row_group_with_null() { - let reader = Int64Case { - null_values: 2, - no_null_values_start: 4, - no_null_values_end: 17, - row_per_group: 10, - ..Default::default() - } - .build(); - - Test { - reader: &reader, - // mins are [4, 14] - expected_min: Arc::new(Int64Array::from(vec![4, 14])), - // maxes are [13, 16] - expected_max: Arc::new(Int64Array::from(vec![13, 16])), - // nulls are [0, 2] - expected_null_counts: UInt64Array::from(vec![0, 2]), - // row counts are [10, 5] - expected_row_counts: Some(UInt64Array::from(vec![10, 5])), - column_name: "i64", - check: Check::Both, - } - .run() -} - -#[tokio::test] -async fn test_two_row_groups_with_all_nulls_in_one() { - let reader = Int64Case { - null_values: 4, - no_null_values_start: -2, - no_null_values_end: 2, - row_per_group: 5, - ..Default::default() - } - .build(); - - Test { - reader: &reader, - // mins are [-2, null] - expected_min: Arc::new(Int64Array::from(vec![Some(-2), None])), - // maxes are [1, null] - expected_max: Arc::new(Int64Array::from(vec![Some(1), None])), - // nulls are [1, 3] - expected_null_counts: UInt64Array::from(vec![1, 3]), - // row counts are [5, 3] - expected_row_counts: Some(UInt64Array::from(vec![5, 3])), - column_name: "i64", - check: Check::Both, - } - .run() -} - -#[tokio::test] -async fn test_multiple_data_pages_nulls_and_negatives() { - let reader = Int64Case { - null_values: 3, - no_null_values_start: -1, - no_null_values_end: 10, - row_per_group: 20, - // limit page row count to 4 - data_page_row_count_limit: Some(4), - enable_stats: Some(EnabledStatistics::Page), - } - .build(); - - // Data layout looks like this: - // - // page 0: [-1, 0, 1, 2] - // page 1: [3, 4, 5, 6] - // page 2: [7, 8, 9, null] - // page 3: [null, null] - Test { - reader: &reader, - expected_min: Arc::new(Int64Array::from(vec![Some(-1), Some(3), Some(7), None])), - expected_max: Arc::new(Int64Array::from(vec![Some(2), Some(6), Some(9), None])), - expected_null_counts: UInt64Array::from(vec![0, 0, 1, 2]), - expected_row_counts: Some(UInt64Array::from(vec![4, 4, 4, 2])), - column_name: "i64", - check: Check::DataPage, - } - .run() -} - -#[tokio::test] -async fn test_data_page_stats_with_all_null_page() { - for data_type in &[ - DataType::Boolean, - DataType::UInt64, - DataType::UInt32, - DataType::UInt16, - DataType::UInt8, - DataType::Int64, - DataType::Int32, - DataType::Int16, - DataType::Int8, - DataType::Float16, - DataType::Float32, - DataType::Float64, - DataType::Date32, - DataType::Date64, - DataType::Time32(TimeUnit::Millisecond), - DataType::Time32(TimeUnit::Second), - DataType::Time64(TimeUnit::Microsecond), - DataType::Time64(TimeUnit::Nanosecond), - DataType::Timestamp(TimeUnit::Second, None), - DataType::Timestamp(TimeUnit::Millisecond, None), - DataType::Timestamp(TimeUnit::Microsecond, None), - DataType::Timestamp(TimeUnit::Nanosecond, None), - DataType::Binary, - DataType::LargeBinary, - DataType::FixedSizeBinary(3), - DataType::Utf8, - DataType::LargeUtf8, - DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), - DataType::Decimal128(8, 2), // as INT32 - DataType::Decimal128(10, 2), // as INT64 - DataType::Decimal128(20, 2), // as FIXED_LEN_BYTE_ARRAY - DataType::Decimal256(8, 2), // as INT32 - DataType::Decimal256(10, 2), // as INT64 - DataType::Decimal256(20, 2), // as FIXED_LEN_BYTE_ARRAY - ] { - let batch = - RecordBatch::try_from_iter(vec![("col", new_null_array(data_type, 4))]) - .expect("record batch creation"); - - let reader = - build_parquet_file(4, Some(EnabledStatistics::Page), Some(4), vec![batch]); - - let expected_data_type = match data_type { - DataType::Dictionary(_, value_type) => value_type.as_ref(), - _ => data_type, - }; - - // There is one data page with 4 nulls - // The statistics should be present but null - Test { - reader: &reader, - expected_min: new_null_array(expected_data_type, 1), - expected_max: new_null_array(expected_data_type, 1), - expected_null_counts: UInt64Array::from(vec![4]), - expected_row_counts: Some(UInt64Array::from(vec![4])), - column_name: "col", - check: Check::DataPage, - } - .run() - } -} - -/////////////// MORE GENERAL TESTS ////////////////////// -// . Many columns in a file -// . Differnet data types -// . Different row group sizes - -// Four different integer types -#[tokio::test] -async fn test_int_64() { - // This creates a parquet files of 4 columns named "i8", "i16", "i32", "i64" - let reader = TestReader { - scenario: Scenario::Int, - row_per_group: 5, - } - .build() - .await; - - // since each row has only one data page, the statistics are the same - Test { - reader: &reader, - // mins are [-5, -4, 0, 5] - expected_min: Arc::new(Int64Array::from(vec![-5, -4, 0, 5])), - // maxes are [-1, 0, 4, 9] - expected_max: Arc::new(Int64Array::from(vec![-1, 0, 4, 9])), - // nulls are [0, 0, 0, 0] - expected_null_counts: UInt64Array::from(vec![0, 0, 0, 0]), - // row counts are [5, 5, 5, 5] - expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5, 5])), - column_name: "i64", - check: Check::Both, - } - .run(); -} - -#[tokio::test] -async fn test_int_32() { - // This creates a parquet files of 4 columns named "i8", "i16", "i32", "i64" - let reader = TestReader { - scenario: Scenario::Int, - row_per_group: 5, - } - .build() - .await; - - Test { - reader: &reader, - // mins are [-5, -4, 0, 5] - expected_min: Arc::new(Int32Array::from(vec![-5, -4, 0, 5])), - // maxes are [-1, 0, 4, 9] - expected_max: Arc::new(Int32Array::from(vec![-1, 0, 4, 9])), - // nulls are [0, 0, 0, 0] - expected_null_counts: UInt64Array::from(vec![0, 0, 0, 0]), - // row counts are [5, 5, 5, 5] - expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5, 5])), - column_name: "i32", - check: Check::Both, - } - .run(); -} - -#[tokio::test] -async fn test_int_16() { - // This creates a parquet files of 4 columns named "i8", "i16", "i32", "i64" - let reader = TestReader { - scenario: Scenario::Int, - row_per_group: 5, - } - .build() - .await; - - Test { - reader: &reader, - // mins are [-5, -4, 0, 5] - expected_min: Arc::new(Int16Array::from(vec![-5, -4, 0, 5])), // panic here because the actual data is Int32Array - // maxes are [-1, 0, 4, 9] - expected_max: Arc::new(Int16Array::from(vec![-1, 0, 4, 9])), - // nulls are [0, 0, 0, 0] - expected_null_counts: UInt64Array::from(vec![0, 0, 0, 0]), - // row counts are [5, 5, 5, 5] - expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5, 5])), - column_name: "i16", - check: Check::Both, - } - .run(); -} - -#[tokio::test] -async fn test_int_8() { - // This creates a parquet files of 4 columns named "i8", "i16", "i32", "i64" - let reader = TestReader { - scenario: Scenario::Int, - row_per_group: 5, - } - .build() - .await; - - Test { - reader: &reader, - // mins are [-5, -4, 0, 5] - expected_min: Arc::new(Int8Array::from(vec![-5, -4, 0, 5])), // panic here because the actual data is Int32Array - // maxes are [-1, 0, 4, 9] - expected_max: Arc::new(Int8Array::from(vec![-1, 0, 4, 9])), - // nulls are [0, 0, 0, 0] - expected_null_counts: UInt64Array::from(vec![0, 0, 0, 0]), - // row counts are [5, 5, 5, 5] - expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5, 5])), - column_name: "i8", - check: Check::Both, - } - .run(); -} - -#[tokio::test] -async fn test_float_16() { - // This creates a parquet files of 1 column named f - let reader = TestReader { - scenario: Scenario::Float16, - row_per_group: 5, - } - .build() - .await; - - Test { - reader: &reader, - // mins are [-5, -4, 0, 5] - expected_min: Arc::new(Float16Array::from(vec![ - f16::from_f32(-5.), - f16::from_f32(-4.), - f16::from_f32(-0.), - f16::from_f32(5.), - ])), - // maxes are [-1, 0, 4, 9] - expected_max: Arc::new(Float16Array::from(vec![ - f16::from_f32(-1.), - f16::from_f32(0.), - f16::from_f32(4.), - f16::from_f32(9.), - ])), - // nulls are [0, 0, 0, 0] - expected_null_counts: UInt64Array::from(vec![0, 0, 0, 0]), - // row counts are [5, 5, 5, 5] - expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5, 5])), - column_name: "f", - check: Check::Both, - } - .run(); -} - -#[tokio::test] -async fn test_float_32() { - // This creates a parquet files of 1 column named f - let reader = TestReader { - scenario: Scenario::Float32, - row_per_group: 5, - } - .build() - .await; - - Test { - reader: &reader, - // mins are [-5, -4, 0, 5] - expected_min: Arc::new(Float32Array::from(vec![-5., -4., -0., 5.0])), - // maxes are [-1, 0, 4, 9] - expected_max: Arc::new(Float32Array::from(vec![-1., 0., 4., 9.])), - // nulls are [0, 0, 0, 0] - expected_null_counts: UInt64Array::from(vec![0, 0, 0, 0]), - // row counts are [5, 5, 5, 5] - expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5, 5])), - column_name: "f", - check: Check::Both, - } - .run(); -} - -#[tokio::test] -async fn test_float_64() { - // This creates a parquet files of 1 column named f - let reader = TestReader { - scenario: Scenario::Float64, - row_per_group: 5, - } - .build() - .await; - - Test { - reader: &reader, - // mins are [-5, -4, 0, 5] - expected_min: Arc::new(Float64Array::from(vec![-5., -4., -0., 5.0])), - // maxes are [-1, 0, 4, 9] - expected_max: Arc::new(Float64Array::from(vec![-1., 0., 4., 9.])), - // nulls are [0, 0, 0, 0] - expected_null_counts: UInt64Array::from(vec![0, 0, 0, 0]), - // row counts are [5, 5, 5, 5] - expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5, 5])), - column_name: "f", - check: Check::Both, - } - .run(); -} - -// timestamp -#[tokio::test] -async fn test_timestamp() { - // This creates a parquet files of 9 columns named "nanos", "nanos_timezoned", "micros", "micros_timezoned", "millis", "millis_timezoned", "seconds", "seconds_timezoned", "names" - // "nanos" --> TimestampNanosecondArray - // "nanos_timezoned" --> TimestampNanosecondArray - // "micros" --> TimestampMicrosecondArray - // "micros_timezoned" --> TimestampMicrosecondArray - // "millis" --> TimestampMillisecondArray - // "millis_timezoned" --> TimestampMillisecondArray - // "seconds" --> TimestampSecondArray - // "seconds_timezoned" --> TimestampSecondArray - // "names" --> StringArray - // - // The file is created by 4 record batches, each has 5 rows. - // Since the row group size is set to 5, those 4 batches will go into 4 row groups - // This creates a parquet files of 4 columns named "nanos", "nanos_timezoned", "micros", "micros_timezoned", "millis", "millis_timezoned", "seconds", "seconds_timezoned" - let reader = TestReader { - scenario: Scenario::Timestamps, - row_per_group: 5, - } - .build() - .await; - - let tz = "Pacific/Efate"; - - Test { - reader: &reader, - expected_min: Arc::new(TimestampNanosecondArray::from(vec![ - TimestampNanosecondType::parse("2020-01-01T01:01:01"), - TimestampNanosecondType::parse("2020-01-01T01:01:11"), - TimestampNanosecondType::parse("2020-01-01T01:11:01"), - TimestampNanosecondType::parse("2020-01-11T01:01:01"), - ])), - expected_max: Arc::new(TimestampNanosecondArray::from(vec![ - TimestampNanosecondType::parse("2020-01-02T01:01:01"), - TimestampNanosecondType::parse("2020-01-02T01:01:11"), - TimestampNanosecondType::parse("2020-01-02T01:11:01"), - TimestampNanosecondType::parse("2020-01-12T01:01:01"), - ])), - // nulls are [1, 1, 1, 1] - expected_null_counts: UInt64Array::from(vec![1, 1, 1, 1]), - // row counts are [5, 5, 5, 5] - expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5, 5])), - column_name: "nanos", - check: Check::Both, - } - .run(); - - Test { - reader: &reader, - expected_min: Arc::new( - TimestampNanosecondArray::from(vec![ - TimestampNanosecondType::parse("2020-01-01T01:01:01"), - TimestampNanosecondType::parse("2020-01-01T01:01:11"), - TimestampNanosecondType::parse("2020-01-01T01:11:01"), - TimestampNanosecondType::parse("2020-01-11T01:01:01"), - ]) - .with_timezone(tz), - ), - expected_max: Arc::new( - TimestampNanosecondArray::from(vec![ - TimestampNanosecondType::parse("2020-01-02T01:01:01"), - TimestampNanosecondType::parse("2020-01-02T01:01:11"), - TimestampNanosecondType::parse("2020-01-02T01:11:01"), - TimestampNanosecondType::parse("2020-01-12T01:01:01"), - ]) - .with_timezone(tz), - ), - // nulls are [1, 1, 1, 1] - expected_null_counts: UInt64Array::from(vec![1, 1, 1, 1]), - // row counts are [5, 5, 5, 5] - expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5, 5])), - column_name: "nanos_timezoned", - check: Check::Both, - } - .run(); - - // micros - Test { - reader: &reader, - expected_min: Arc::new(TimestampMicrosecondArray::from(vec![ - TimestampMicrosecondType::parse("2020-01-01T01:01:01"), - TimestampMicrosecondType::parse("2020-01-01T01:01:11"), - TimestampMicrosecondType::parse("2020-01-01T01:11:01"), - TimestampMicrosecondType::parse("2020-01-11T01:01:01"), - ])), - expected_max: Arc::new(TimestampMicrosecondArray::from(vec![ - TimestampMicrosecondType::parse("2020-01-02T01:01:01"), - TimestampMicrosecondType::parse("2020-01-02T01:01:11"), - TimestampMicrosecondType::parse("2020-01-02T01:11:01"), - TimestampMicrosecondType::parse("2020-01-12T01:01:01"), - ])), - expected_null_counts: UInt64Array::from(vec![1, 1, 1, 1]), - expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5, 5])), - column_name: "micros", - check: Check::Both, - } - .run(); - - Test { - reader: &reader, - expected_min: Arc::new( - TimestampMicrosecondArray::from(vec![ - TimestampMicrosecondType::parse("2020-01-01T01:01:01"), - TimestampMicrosecondType::parse("2020-01-01T01:01:11"), - TimestampMicrosecondType::parse("2020-01-01T01:11:01"), - TimestampMicrosecondType::parse("2020-01-11T01:01:01"), - ]) - .with_timezone(tz), - ), - expected_max: Arc::new( - TimestampMicrosecondArray::from(vec![ - TimestampMicrosecondType::parse("2020-01-02T01:01:01"), - TimestampMicrosecondType::parse("2020-01-02T01:01:11"), - TimestampMicrosecondType::parse("2020-01-02T01:11:01"), - TimestampMicrosecondType::parse("2020-01-12T01:01:01"), - ]) - .with_timezone(tz), - ), - // nulls are [1, 1, 1, 1] - expected_null_counts: UInt64Array::from(vec![1, 1, 1, 1]), - // row counts are [5, 5, 5, 5] - expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5, 5])), - column_name: "micros_timezoned", - check: Check::Both, - } - .run(); - - // millis - Test { - reader: &reader, - expected_min: Arc::new(TimestampMillisecondArray::from(vec![ - TimestampMillisecondType::parse("2020-01-01T01:01:01"), - TimestampMillisecondType::parse("2020-01-01T01:01:11"), - TimestampMillisecondType::parse("2020-01-01T01:11:01"), - TimestampMillisecondType::parse("2020-01-11T01:01:01"), - ])), - expected_max: Arc::new(TimestampMillisecondArray::from(vec![ - TimestampMillisecondType::parse("2020-01-02T01:01:01"), - TimestampMillisecondType::parse("2020-01-02T01:01:11"), - TimestampMillisecondType::parse("2020-01-02T01:11:01"), - TimestampMillisecondType::parse("2020-01-12T01:01:01"), - ])), - expected_null_counts: UInt64Array::from(vec![1, 1, 1, 1]), - expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5, 5])), - column_name: "millis", - check: Check::Both, - } - .run(); - - Test { - reader: &reader, - expected_min: Arc::new( - TimestampMillisecondArray::from(vec![ - TimestampMillisecondType::parse("2020-01-01T01:01:01"), - TimestampMillisecondType::parse("2020-01-01T01:01:11"), - TimestampMillisecondType::parse("2020-01-01T01:11:01"), - TimestampMillisecondType::parse("2020-01-11T01:01:01"), - ]) - .with_timezone(tz), - ), - expected_max: Arc::new( - TimestampMillisecondArray::from(vec![ - TimestampMillisecondType::parse("2020-01-02T01:01:01"), - TimestampMillisecondType::parse("2020-01-02T01:01:11"), - TimestampMillisecondType::parse("2020-01-02T01:11:01"), - TimestampMillisecondType::parse("2020-01-12T01:01:01"), - ]) - .with_timezone(tz), - ), - // nulls are [1, 1, 1, 1] - expected_null_counts: UInt64Array::from(vec![1, 1, 1, 1]), - // row counts are [5, 5, 5, 5] - expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5, 5])), - column_name: "millis_timezoned", - check: Check::Both, - } - .run(); - - // seconds - Test { - reader: &reader, - expected_min: Arc::new(TimestampSecondArray::from(vec![ - TimestampSecondType::parse("2020-01-01T01:01:01"), - TimestampSecondType::parse("2020-01-01T01:01:11"), - TimestampSecondType::parse("2020-01-01T01:11:01"), - TimestampSecondType::parse("2020-01-11T01:01:01"), - ])), - expected_max: Arc::new(TimestampSecondArray::from(vec![ - TimestampSecondType::parse("2020-01-02T01:01:01"), - TimestampSecondType::parse("2020-01-02T01:01:11"), - TimestampSecondType::parse("2020-01-02T01:11:01"), - TimestampSecondType::parse("2020-01-12T01:01:01"), - ])), - expected_null_counts: UInt64Array::from(vec![1, 1, 1, 1]), - expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5, 5])), - column_name: "seconds", - check: Check::Both, - } - .run(); - - Test { - reader: &reader, - expected_min: Arc::new( - TimestampSecondArray::from(vec![ - TimestampSecondType::parse("2020-01-01T01:01:01"), - TimestampSecondType::parse("2020-01-01T01:01:11"), - TimestampSecondType::parse("2020-01-01T01:11:01"), - TimestampSecondType::parse("2020-01-11T01:01:01"), - ]) - .with_timezone(tz), - ), - expected_max: Arc::new( - TimestampSecondArray::from(vec![ - TimestampSecondType::parse("2020-01-02T01:01:01"), - TimestampSecondType::parse("2020-01-02T01:01:11"), - TimestampSecondType::parse("2020-01-02T01:11:01"), - TimestampSecondType::parse("2020-01-12T01:01:01"), - ]) - .with_timezone(tz), - ), - // nulls are [1, 1, 1, 1] - expected_null_counts: UInt64Array::from(vec![1, 1, 1, 1]), - // row counts are [5, 5, 5, 5] - expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5, 5])), - column_name: "seconds_timezoned", - check: Check::Both, - } - .run(); -} - -// timestamp with different row group sizes -#[tokio::test] -async fn test_timestamp_diff_rg_sizes() { - // This creates a parquet files of 9 columns named "nanos", "nanos_timezoned", "micros", "micros_timezoned", "millis", "millis_timezoned", "seconds", "seconds_timezoned", "names" - // "nanos" --> TimestampNanosecondArray - // "nanos_timezoned" --> TimestampNanosecondArray - // "micros" --> TimestampMicrosecondArray - // "micros_timezoned" --> TimestampMicrosecondArray - // "millis" --> TimestampMillisecondArray - // "millis_timezoned" --> TimestampMillisecondArray - // "seconds" --> TimestampSecondArray - // "seconds_timezoned" --> TimestampSecondArray - // "names" --> StringArray - // - // The file is created by 4 record batches (each has a null row), each has 5 rows but then will be split into 3 row groups with size 8, 8, 4 - let reader = TestReader { - scenario: Scenario::Timestamps, - row_per_group: 8, // note that the row group size is 8 - } - .build() - .await; - - let tz = "Pacific/Efate"; - - Test { - reader: &reader, - expected_min: Arc::new(TimestampNanosecondArray::from(vec![ - TimestampNanosecondType::parse("2020-01-01T01:01:01"), - TimestampNanosecondType::parse("2020-01-01T01:11:01"), - TimestampNanosecondType::parse("2020-01-11T01:02:01"), - ])), - expected_max: Arc::new(TimestampNanosecondArray::from(vec![ - TimestampNanosecondType::parse("2020-01-02T01:01:01"), - TimestampNanosecondType::parse("2020-01-11T01:01:01"), - TimestampNanosecondType::parse("2020-01-12T01:01:01"), - ])), - // nulls are [1, 2, 1] - expected_null_counts: UInt64Array::from(vec![1, 2, 1]), - // row counts are [8, 8, 4] - expected_row_counts: Some(UInt64Array::from(vec![8, 8, 4])), - column_name: "nanos", - check: Check::Both, - } - .run(); - - Test { - reader: &reader, - expected_min: Arc::new( - TimestampNanosecondArray::from(vec![ - TimestampNanosecondType::parse("2020-01-01T01:01:01"), - TimestampNanosecondType::parse("2020-01-01T01:11:01"), - TimestampNanosecondType::parse("2020-01-11T01:02:01"), - ]) - .with_timezone(tz), - ), - expected_max: Arc::new( - TimestampNanosecondArray::from(vec![ - TimestampNanosecondType::parse("2020-01-02T01:01:01"), - TimestampNanosecondType::parse("2020-01-11T01:01:01"), - TimestampNanosecondType::parse("2020-01-12T01:01:01"), - ]) - .with_timezone(tz), - ), - // nulls are [1, 2, 1] - expected_null_counts: UInt64Array::from(vec![1, 2, 1]), - // row counts are [8, 8, 4] - expected_row_counts: Some(UInt64Array::from(vec![8, 8, 4])), - column_name: "nanos_timezoned", - check: Check::Both, - } - .run(); - - // micros - Test { - reader: &reader, - expected_min: Arc::new(TimestampMicrosecondArray::from(vec![ - TimestampMicrosecondType::parse("2020-01-01T01:01:01"), - TimestampMicrosecondType::parse("2020-01-01T01:11:01"), - TimestampMicrosecondType::parse("2020-01-11T01:02:01"), - ])), - expected_max: Arc::new(TimestampMicrosecondArray::from(vec![ - TimestampMicrosecondType::parse("2020-01-02T01:01:01"), - TimestampMicrosecondType::parse("2020-01-11T01:01:01"), - TimestampMicrosecondType::parse("2020-01-12T01:01:01"), - ])), - expected_null_counts: UInt64Array::from(vec![1, 2, 1]), - expected_row_counts: Some(UInt64Array::from(vec![8, 8, 4])), - column_name: "micros", - check: Check::Both, - } - .run(); - - Test { - reader: &reader, - expected_min: Arc::new( - TimestampMicrosecondArray::from(vec![ - TimestampMicrosecondType::parse("2020-01-01T01:01:01"), - TimestampMicrosecondType::parse("2020-01-01T01:11:01"), - TimestampMicrosecondType::parse("2020-01-11T01:02:01"), - ]) - .with_timezone(tz), - ), - expected_max: Arc::new( - TimestampMicrosecondArray::from(vec![ - TimestampMicrosecondType::parse("2020-01-02T01:01:01"), - TimestampMicrosecondType::parse("2020-01-11T01:01:01"), - TimestampMicrosecondType::parse("2020-01-12T01:01:01"), - ]) - .with_timezone(tz), - ), - // nulls are [1, 2, 1] - expected_null_counts: UInt64Array::from(vec![1, 2, 1]), - // row counts are [8, 8, 4] - expected_row_counts: Some(UInt64Array::from(vec![8, 8, 4])), - column_name: "micros_timezoned", - check: Check::Both, - } - .run(); - - // millis - Test { - reader: &reader, - expected_min: Arc::new(TimestampMillisecondArray::from(vec![ - TimestampMillisecondType::parse("2020-01-01T01:01:01"), - TimestampMillisecondType::parse("2020-01-01T01:11:01"), - TimestampMillisecondType::parse("2020-01-11T01:02:01"), - ])), - expected_max: Arc::new(TimestampMillisecondArray::from(vec![ - TimestampMillisecondType::parse("2020-01-02T01:01:01"), - TimestampMillisecondType::parse("2020-01-11T01:01:01"), - TimestampMillisecondType::parse("2020-01-12T01:01:01"), - ])), - expected_null_counts: UInt64Array::from(vec![1, 2, 1]), - expected_row_counts: Some(UInt64Array::from(vec![8, 8, 4])), - column_name: "millis", - check: Check::Both, - } - .run(); - - Test { - reader: &reader, - expected_min: Arc::new( - TimestampMillisecondArray::from(vec![ - TimestampMillisecondType::parse("2020-01-01T01:01:01"), - TimestampMillisecondType::parse("2020-01-01T01:11:01"), - TimestampMillisecondType::parse("2020-01-11T01:02:01"), - ]) - .with_timezone(tz), - ), - expected_max: Arc::new( - TimestampMillisecondArray::from(vec![ - TimestampMillisecondType::parse("2020-01-02T01:01:01"), - TimestampMillisecondType::parse("2020-01-11T01:01:01"), - TimestampMillisecondType::parse("2020-01-12T01:01:01"), - ]) - .with_timezone(tz), - ), - // nulls are [1, 2, 1] - expected_null_counts: UInt64Array::from(vec![1, 2, 1]), - // row counts are [8, 8, 4] - expected_row_counts: Some(UInt64Array::from(vec![8, 8, 4])), - column_name: "millis_timezoned", - check: Check::Both, - } - .run(); - - // seconds - Test { - reader: &reader, - expected_min: Arc::new(TimestampSecondArray::from(vec![ - TimestampSecondType::parse("2020-01-01T01:01:01"), - TimestampSecondType::parse("2020-01-01T01:11:01"), - TimestampSecondType::parse("2020-01-11T01:02:01"), - ])), - expected_max: Arc::new(TimestampSecondArray::from(vec![ - TimestampSecondType::parse("2020-01-02T01:01:01"), - TimestampSecondType::parse("2020-01-11T01:01:01"), - TimestampSecondType::parse("2020-01-12T01:01:01"), - ])), - expected_null_counts: UInt64Array::from(vec![1, 2, 1]), - expected_row_counts: Some(UInt64Array::from(vec![8, 8, 4])), - column_name: "seconds", - check: Check::Both, - } - .run(); - - Test { - reader: &reader, - expected_min: Arc::new( - TimestampSecondArray::from(vec![ - TimestampSecondType::parse("2020-01-01T01:01:01"), - TimestampSecondType::parse("2020-01-01T01:11:01"), - TimestampSecondType::parse("2020-01-11T01:02:01"), - ]) - .with_timezone(tz), - ), - expected_max: Arc::new( - TimestampSecondArray::from(vec![ - TimestampSecondType::parse("2020-01-02T01:01:01"), - TimestampSecondType::parse("2020-01-11T01:01:01"), - TimestampSecondType::parse("2020-01-12T01:01:01"), - ]) - .with_timezone(tz), - ), - // nulls are [1, 2, 1] - expected_null_counts: UInt64Array::from(vec![1, 2, 1]), - // row counts are [8, 8, 4] - expected_row_counts: Some(UInt64Array::from(vec![8, 8, 4])), - column_name: "seconds_timezoned", - check: Check::Both, - } - .run(); -} - -// date with different row group sizes -#[tokio::test] -async fn test_dates_32_diff_rg_sizes() { - // This creates a parquet files of 3 columns named "date32", "date64", "names" - // "date32" --> Date32Array - // "date64" --> Date64Array - // "names" --> StringArray - // - // The file is created by 4 record batches (each has a null row), each has 5 rows but then will be split into 2 row groups with size 13, 7 - let reader = TestReader { - scenario: Scenario::Dates, - row_per_group: 13, - } - .build() - .await; - - Test { - reader: &reader, - // mins are [2020-01-01, 2020-10-30] - expected_min: Arc::new(Date32Array::from(vec![ - Date32Type::parse("2020-01-01"), - Date32Type::parse("2020-10-30"), - ])), - // maxes are [2020-10-29, 2029-11-12] - expected_max: Arc::new(Date32Array::from(vec![ - Date32Type::parse("2020-10-29"), - Date32Type::parse("2029-11-12"), - ])), - // nulls are [2, 2] - expected_null_counts: UInt64Array::from(vec![2, 2]), - // row counts are [13, 7] - expected_row_counts: Some(UInt64Array::from(vec![13, 7])), - column_name: "date32", - check: Check::Both, - } - .run(); -} - -#[tokio::test] -async fn test_time32_second_diff_rg_sizes() { - let reader = TestReader { - scenario: Scenario::Time32Second, - row_per_group: 4, - } - .build() - .await; - - // Test for Time32Second column - Test { - reader: &reader, - // Assuming specific minimum and maximum values for demonstration - expected_min: Arc::new(Time32SecondArray::from(vec![18506, 18510, 18514, 18518])), - expected_max: Arc::new(Time32SecondArray::from(vec![18509, 18513, 18517, 18521])), - expected_null_counts: UInt64Array::from(vec![0, 0, 0, 0]), // Assuming 1 null per row group for simplicity - expected_row_counts: Some(UInt64Array::from(vec![4, 4, 4, 4])), - column_name: "second", - check: Check::Both, - } - .run(); -} - -#[tokio::test] -async fn test_time32_millisecond_diff_rg_sizes() { - let reader = TestReader { - scenario: Scenario::Time32Millisecond, - row_per_group: 4, - } - .build() - .await; - - // Test for Time32Millisecond column - Test { - reader: &reader, - // Assuming specific minimum and maximum values for demonstration - expected_min: Arc::new(Time32MillisecondArray::from(vec![ - 3600000, 3600004, 3600008, 3600012, - ])), - expected_max: Arc::new(Time32MillisecondArray::from(vec![ - 3600003, 3600007, 3600011, 3600015, - ])), - expected_null_counts: UInt64Array::from(vec![0, 0, 0, 0]), // Assuming 1 null per row group for simplicity - expected_row_counts: Some(UInt64Array::from(vec![4, 4, 4, 4])), - column_name: "millisecond", - check: Check::Both, - } - .run(); -} - -#[tokio::test] -async fn test_time64_microsecond_diff_rg_sizes() { - let reader = TestReader { - scenario: Scenario::Time64Microsecond, - row_per_group: 4, - } - .build() - .await; - - // Test for Time64MicroSecond column - Test { - reader: &reader, - // Assuming specific minimum and maximum values for demonstration - expected_min: Arc::new(Time64MicrosecondArray::from(vec![ - 1234567890123, - 1234567890127, - 1234567890131, - 1234567890135, - ])), - expected_max: Arc::new(Time64MicrosecondArray::from(vec![ - 1234567890126, - 1234567890130, - 1234567890134, - 1234567890138, - ])), - expected_null_counts: UInt64Array::from(vec![0, 0, 0, 0]), // Assuming 1 null per row group for simplicity - expected_row_counts: Some(UInt64Array::from(vec![4, 4, 4, 4])), - column_name: "microsecond", - check: Check::Both, - } - .run(); -} - -#[tokio::test] -async fn test_time64_nanosecond_diff_rg_sizes() { - let reader = TestReader { - scenario: Scenario::Time64Nanosecond, - row_per_group: 4, - } - .build() - .await; - - // Test for Time32Second column - Test { - reader: &reader, - // Assuming specific minimum and maximum values for demonstration - expected_min: Arc::new(Time64NanosecondArray::from(vec![ - 987654321012345, - 987654321012349, - 987654321012353, - 987654321012357, - ])), - expected_max: Arc::new(Time64NanosecondArray::from(vec![ - 987654321012348, - 987654321012352, - 987654321012356, - 987654321012360, - ])), - expected_null_counts: UInt64Array::from(vec![0, 0, 0, 0]), // Assuming 1 null per row group for simplicity - expected_row_counts: Some(UInt64Array::from(vec![4, 4, 4, 4])), - column_name: "nanosecond", - check: Check::Both, - } - .run(); -} - -#[tokio::test] -async fn test_dates_64_diff_rg_sizes() { - // The file is created by 4 record batches (each has a null row), each has 5 rows but then will be split into 2 row groups with size 13, 7 - let reader = TestReader { - scenario: Scenario::Dates, - row_per_group: 13, - } - .build() - .await; - Test { - reader: &reader, - expected_min: Arc::new(Date64Array::from(vec![ - Date64Type::parse("2020-01-01"), - Date64Type::parse("2020-10-30"), - ])), - expected_max: Arc::new(Date64Array::from(vec![ - Date64Type::parse("2020-10-29"), - Date64Type::parse("2029-11-12"), - ])), - expected_null_counts: UInt64Array::from(vec![2, 2]), - expected_row_counts: Some(UInt64Array::from(vec![13, 7])), - column_name: "date64", - check: Check::Both, - } - .run(); -} - -#[tokio::test] -async fn test_uint() { - // This creates a parquet files of 4 columns named "u8", "u16", "u32", "u64" - // "u8" --> UInt8Array - // "u16" --> UInt16Array - // "u32" --> UInt32Array - // "u64" --> UInt64Array - - // The file is created by 4 record batches (each has a null row), each has 5 rows but then will be split into 5 row groups with size 4 - let reader = TestReader { - scenario: Scenario::UInt, - row_per_group: 4, - } - .build() - .await; - - Test { - reader: &reader, - expected_min: Arc::new(UInt8Array::from(vec![0, 1, 4, 7, 251])), - expected_max: Arc::new(UInt8Array::from(vec![3, 4, 6, 250, 254])), - expected_null_counts: UInt64Array::from(vec![0, 0, 0, 0, 0]), - expected_row_counts: Some(UInt64Array::from(vec![4, 4, 4, 4, 4])), - column_name: "u8", - check: Check::Both, - } - .run(); - - Test { - reader: &reader, - expected_min: Arc::new(UInt16Array::from(vec![0, 1, 4, 7, 251])), - expected_max: Arc::new(UInt16Array::from(vec![3, 4, 6, 250, 254])), - expected_null_counts: UInt64Array::from(vec![0, 0, 0, 0, 0]), - expected_row_counts: Some(UInt64Array::from(vec![4, 4, 4, 4, 4])), - column_name: "u16", - check: Check::Both, - } - .run(); - - Test { - reader: &reader, - expected_min: Arc::new(UInt32Array::from(vec![0, 1, 4, 7, 251])), - expected_max: Arc::new(UInt32Array::from(vec![3, 4, 6, 250, 254])), - expected_null_counts: UInt64Array::from(vec![0, 0, 0, 0, 0]), - expected_row_counts: Some(UInt64Array::from(vec![4, 4, 4, 4, 4])), - column_name: "u32", - check: Check::Both, - } - .run(); - - Test { - reader: &reader, - expected_min: Arc::new(UInt64Array::from(vec![0, 1, 4, 7, 251])), - expected_max: Arc::new(UInt64Array::from(vec![3, 4, 6, 250, 254])), - expected_null_counts: UInt64Array::from(vec![0, 0, 0, 0, 0]), - expected_row_counts: Some(UInt64Array::from(vec![4, 4, 4, 4, 4])), - column_name: "u64", - check: Check::Both, - } - .run(); -} - -#[tokio::test] -async fn test_int32_range() { - // This creates a parquet file of 1 column "i" - // file has 2 record batches, each has 2 rows. They will be saved into one row group - let reader = TestReader { - scenario: Scenario::Int32Range, - row_per_group: 5, - } - .build() - .await; - - Test { - reader: &reader, - expected_min: Arc::new(Int32Array::from(vec![0])), - expected_max: Arc::new(Int32Array::from(vec![300000])), - expected_null_counts: UInt64Array::from(vec![0]), - expected_row_counts: Some(UInt64Array::from(vec![4])), - column_name: "i", - check: Check::Both, - } - .run(); -} - -#[tokio::test] -async fn test_uint32_range() { - // This creates a parquet file of 1 column "u" - // file has 2 record batches, each has 2 rows. They will be saved into one row group - let reader = TestReader { - scenario: Scenario::UInt32Range, - row_per_group: 5, - } - .build() - .await; - - Test { - reader: &reader, - expected_min: Arc::new(UInt32Array::from(vec![0])), - expected_max: Arc::new(UInt32Array::from(vec![300000])), - expected_null_counts: UInt64Array::from(vec![0]), - expected_row_counts: Some(UInt64Array::from(vec![4])), - column_name: "u", - check: Check::Both, - } - .run(); -} - -#[tokio::test] -async fn test_numeric_limits_unsigned() { - // file has 7 rows, 2 row groups: one with 5 rows, one with 2 rows. - let reader = TestReader { - scenario: Scenario::NumericLimits, - row_per_group: 5, - } - .build() - .await; - - Test { - reader: &reader, - expected_min: Arc::new(UInt8Array::from(vec![u8::MIN, 100])), - expected_max: Arc::new(UInt8Array::from(vec![100, u8::MAX])), - expected_null_counts: UInt64Array::from(vec![0, 0]), - expected_row_counts: Some(UInt64Array::from(vec![5, 2])), - column_name: "u8", - check: Check::Both, - } - .run(); - - Test { - reader: &reader, - expected_min: Arc::new(UInt16Array::from(vec![u16::MIN, 100])), - expected_max: Arc::new(UInt16Array::from(vec![100, u16::MAX])), - expected_null_counts: UInt64Array::from(vec![0, 0]), - expected_row_counts: Some(UInt64Array::from(vec![5, 2])), - column_name: "u16", - check: Check::Both, - } - .run(); - - Test { - reader: &reader, - expected_min: Arc::new(UInt32Array::from(vec![u32::MIN, 100])), - expected_max: Arc::new(UInt32Array::from(vec![100, u32::MAX])), - expected_null_counts: UInt64Array::from(vec![0, 0]), - expected_row_counts: Some(UInt64Array::from(vec![5, 2])), - column_name: "u32", - check: Check::Both, - } - .run(); - - Test { - reader: &reader, - expected_min: Arc::new(UInt64Array::from(vec![u64::MIN, 100])), - expected_max: Arc::new(UInt64Array::from(vec![100, u64::MAX])), - expected_null_counts: UInt64Array::from(vec![0, 0]), - expected_row_counts: Some(UInt64Array::from(vec![5, 2])), - column_name: "u64", - check: Check::Both, - } - .run(); -} - -#[tokio::test] -async fn test_numeric_limits_signed() { - // file has 7 rows, 2 row groups: one with 5 rows, one with 2 rows. - let reader = TestReader { - scenario: Scenario::NumericLimits, - row_per_group: 5, - } - .build() - .await; - - Test { - reader: &reader, - expected_min: Arc::new(Int8Array::from(vec![i8::MIN, -100])), - expected_max: Arc::new(Int8Array::from(vec![100, i8::MAX])), - expected_null_counts: UInt64Array::from(vec![0, 0]), - expected_row_counts: Some(UInt64Array::from(vec![5, 2])), - column_name: "i8", - check: Check::Both, - } - .run(); - - Test { - reader: &reader, - expected_min: Arc::new(Int16Array::from(vec![i16::MIN, -100])), - expected_max: Arc::new(Int16Array::from(vec![100, i16::MAX])), - expected_null_counts: UInt64Array::from(vec![0, 0]), - expected_row_counts: Some(UInt64Array::from(vec![5, 2])), - column_name: "i16", - check: Check::Both, - } - .run(); - - Test { - reader: &reader, - expected_min: Arc::new(Int32Array::from(vec![i32::MIN, -100])), - expected_max: Arc::new(Int32Array::from(vec![100, i32::MAX])), - expected_null_counts: UInt64Array::from(vec![0, 0]), - expected_row_counts: Some(UInt64Array::from(vec![5, 2])), - column_name: "i32", - check: Check::Both, - } - .run(); - - Test { - reader: &reader, - expected_min: Arc::new(Int64Array::from(vec![i64::MIN, -100])), - expected_max: Arc::new(Int64Array::from(vec![100, i64::MAX])), - expected_null_counts: UInt64Array::from(vec![0, 0]), - expected_row_counts: Some(UInt64Array::from(vec![5, 2])), - column_name: "i64", - check: Check::Both, - } - .run(); -} - -#[tokio::test] -async fn test_numeric_limits_float() { - // file has 7 rows, 2 row groups: one with 5 rows, one with 2 rows. - let reader = TestReader { - scenario: Scenario::NumericLimits, - row_per_group: 5, - } - .build() - .await; - - Test { - reader: &reader, - expected_min: Arc::new(Float32Array::from(vec![f32::MIN, -100.0])), - expected_max: Arc::new(Float32Array::from(vec![100.0, f32::MAX])), - expected_null_counts: UInt64Array::from(vec![0, 0]), - expected_row_counts: Some(UInt64Array::from(vec![5, 2])), - column_name: "f32", - check: Check::Both, - } - .run(); - - Test { - reader: &reader, - expected_min: Arc::new(Float64Array::from(vec![f64::MIN, -100.0])), - expected_max: Arc::new(Float64Array::from(vec![100.0, f64::MAX])), - expected_null_counts: UInt64Array::from(vec![0, 0]), - expected_row_counts: Some(UInt64Array::from(vec![5, 2])), - column_name: "f64", - check: Check::Both, - } - .run(); - - Test { - reader: &reader, - expected_min: Arc::new(Float32Array::from(vec![-1.0, -100.0])), - expected_max: Arc::new(Float32Array::from(vec![100.0, -100.0])), - expected_null_counts: UInt64Array::from(vec![0, 0]), - expected_row_counts: Some(UInt64Array::from(vec![5, 2])), - column_name: "f32_nan", - check: Check::Both, - } - .run(); - - Test { - reader: &reader, - expected_min: Arc::new(Float64Array::from(vec![-1.0, -100.0])), - expected_max: Arc::new(Float64Array::from(vec![100.0, -100.0])), - expected_null_counts: UInt64Array::from(vec![0, 0]), - expected_row_counts: Some(UInt64Array::from(vec![5, 2])), - column_name: "f64_nan", - check: Check::Both, - } - .run(); -} - -#[tokio::test] -async fn test_float64() { - // This creates a parquet file of 1 column "f" - // file has 4 record batches, each has 5 rows. They will be saved into 4 row groups - let reader = TestReader { - scenario: Scenario::Float64, - row_per_group: 5, - } - .build() - .await; - - Test { - reader: &reader, - expected_min: Arc::new(Float64Array::from(vec![-5.0, -4.0, -0.0, 5.0])), - expected_max: Arc::new(Float64Array::from(vec![-1.0, 0.0, 4.0, 9.0])), - expected_null_counts: UInt64Array::from(vec![0, 0, 0, 0]), - expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5, 5])), - column_name: "f", - check: Check::Both, - } - .run(); -} - -#[tokio::test] -async fn test_float16() { - // This creates a parquet file of 1 column "f" - // file has 4 record batches, each has 5 rows. They will be saved into 4 row groups - let reader = TestReader { - scenario: Scenario::Float16, - row_per_group: 5, - } - .build() - .await; - - Test { - reader: &reader, - expected_min: Arc::new(Float16Array::from( - vec![-5.0, -4.0, -0.0, 5.0] - .into_iter() - .map(f16::from_f32) - .collect::>(), - )), - expected_max: Arc::new(Float16Array::from( - vec![-1.0, 0.0, 4.0, 9.0] - .into_iter() - .map(f16::from_f32) - .collect::>(), - )), - expected_null_counts: UInt64Array::from(vec![0, 0, 0, 0]), - expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5, 5])), - column_name: "f", - check: Check::Both, - } - .run(); -} - -#[tokio::test] -async fn test_decimal() { - // This creates a parquet file of 1 column "decimal_col" with decimal data type and precicion 9, scale 2 - // file has 3 record batches, each has 5 rows. They will be saved into 3 row groups - let reader = TestReader { - scenario: Scenario::Decimal, - row_per_group: 5, - } - .build() - .await; - - Test { - reader: &reader, - expected_min: Arc::new( - Decimal128Array::from(vec![100, -500, 2000]) - .with_precision_and_scale(9, 2) - .unwrap(), - ), - expected_max: Arc::new( - Decimal128Array::from(vec![600, 600, 6000]) - .with_precision_and_scale(9, 2) - .unwrap(), - ), - expected_null_counts: UInt64Array::from(vec![0, 0, 0]), - expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5])), - column_name: "decimal_col", - check: Check::Both, - } - .run(); -} -#[tokio::test] -async fn test_decimal_256() { - // This creates a parquet file of 1 column "decimal256_col" with decimal data type and precicion 9, scale 2 - // file has 3 record batches, each has 5 rows. They will be saved into 3 row groups - let reader = TestReader { - scenario: Scenario::Decimal256, - row_per_group: 5, - } - .build() - .await; - - Test { - reader: &reader, - expected_min: Arc::new( - Decimal256Array::from(vec![ - i256::from(100), - i256::from(-500), - i256::from(2000), - ]) - .with_precision_and_scale(9, 2) - .unwrap(), - ), - expected_max: Arc::new( - Decimal256Array::from(vec![ - i256::from(600), - i256::from(600), - i256::from(6000), - ]) - .with_precision_and_scale(9, 2) - .unwrap(), - ), - expected_null_counts: UInt64Array::from(vec![0, 0, 0]), - expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5])), - column_name: "decimal256_col", - check: Check::Both, - } - .run(); -} -#[tokio::test] -async fn test_dictionary() { - let reader = TestReader { - scenario: Scenario::Dictionary, - row_per_group: 5, - } - .build() - .await; - - Test { - reader: &reader, - expected_min: Arc::new(StringArray::from(vec!["abc", "aaa"])), - expected_max: Arc::new(StringArray::from(vec!["def", "fffff"])), - expected_null_counts: UInt64Array::from(vec![1, 0]), - expected_row_counts: Some(UInt64Array::from(vec![5, 2])), - column_name: "string_dict_i8", - check: Check::Both, - } - .run(); - - Test { - reader: &reader, - expected_min: Arc::new(StringArray::from(vec!["abc", "aaa"])), - expected_max: Arc::new(StringArray::from(vec!["def", "fffff"])), - expected_null_counts: UInt64Array::from(vec![1, 0]), - expected_row_counts: Some(UInt64Array::from(vec![5, 2])), - column_name: "string_dict_i32", - check: Check::Both, - } - .run(); - - Test { - reader: &reader, - expected_min: Arc::new(Int64Array::from(vec![-100, 0])), - expected_max: Arc::new(Int64Array::from(vec![0, 100])), - expected_null_counts: UInt64Array::from(vec![1, 0]), - expected_row_counts: Some(UInt64Array::from(vec![5, 2])), - column_name: "int_dict_i8", - check: Check::Both, - } - .run(); -} - -#[tokio::test] -async fn test_byte() { - // This creates a parquet file of 5 columns - // "name" - // "service_string" - // "service_binary" - // "service_fixedsize" - // "service_large_binary" - - // file has 3 record batches, each has 5 rows. They will be saved into 3 row groups - let reader = TestReader { - scenario: Scenario::ByteArray, - row_per_group: 5, - } - .build() - .await; - - // column "name" - Test { - reader: &reader, - expected_min: Arc::new(StringArray::from(vec![ - "all frontends", - "mixed", - "all backends", - ])), - expected_max: Arc::new(StringArray::from(vec![ - "all frontends", - "mixed", - "all backends", - ])), - expected_null_counts: UInt64Array::from(vec![0, 0, 0]), - expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5])), - column_name: "name", - check: Check::Both, - } - .run(); - - // column "service_string" - Test { - reader: &reader, - expected_min: Arc::new(StringArray::from(vec![ - "frontend five", - "backend one", - "backend eight", - ])), - expected_max: Arc::new(StringArray::from(vec![ - "frontend two", - "frontend six", - "backend six", - ])), - expected_null_counts: UInt64Array::from(vec![0, 0, 0]), - expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5])), - column_name: "service_string", - check: Check::Both, - } - .run(); - - // column "service_binary" - - let expected_service_binary_min_values: Vec<&[u8]> = - vec![b"frontend five", b"backend one", b"backend eight"]; - - let expected_service_binary_max_values: Vec<&[u8]> = - vec![b"frontend two", b"frontend six", b"backend six"]; - - Test { - reader: &reader, - expected_min: Arc::new(BinaryArray::from(expected_service_binary_min_values)), - expected_max: Arc::new(BinaryArray::from(expected_service_binary_max_values)), - expected_null_counts: UInt64Array::from(vec![0, 0, 0]), - expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5])), - column_name: "service_binary", - check: Check::Both, - } - .run(); - - // column "service_fixedsize" - // b"fe1", b"be1", b"be4" - let min_input = vec![vec![102, 101, 49], vec![98, 101, 49], vec![98, 101, 52]]; - // b"fe5", b"fe6", b"be8" - let max_input = vec![vec![102, 101, 55], vec![102, 101, 54], vec![98, 101, 56]]; - - Test { - reader: &reader, - expected_min: Arc::new( - FixedSizeBinaryArray::try_from_iter(min_input.into_iter()).unwrap(), - ), - expected_max: Arc::new( - FixedSizeBinaryArray::try_from_iter(max_input.into_iter()).unwrap(), - ), - expected_null_counts: UInt64Array::from(vec![0, 0, 0]), - expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5])), - column_name: "service_fixedsize", - check: Check::Both, - } - .run(); - - let expected_service_large_binary_min_values: Vec<&[u8]> = - vec![b"frontend five", b"backend one", b"backend eight"]; - - let expected_service_large_binary_max_values: Vec<&[u8]> = - vec![b"frontend two", b"frontend six", b"backend six"]; - - Test { - reader: &reader, - expected_min: Arc::new(LargeBinaryArray::from( - expected_service_large_binary_min_values, - )), - expected_max: Arc::new(LargeBinaryArray::from( - expected_service_large_binary_max_values, - )), - expected_null_counts: UInt64Array::from(vec![0, 0, 0]), - expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5])), - column_name: "service_large_binary", - check: Check::Both, - } - .run(); -} - -// PeriodsInColumnNames -#[tokio::test] -async fn test_period_in_column_names() { - // This creates a parquet file of 2 columns "name" and "service.name" - // file has 3 record batches, each has 5 rows. They will be saved into 3 row groups - let reader = TestReader { - scenario: Scenario::PeriodsInColumnNames, - row_per_group: 5, - } - .build() - .await; - - // column "name" - Test { - reader: &reader, - expected_min: Arc::new(StringArray::from(vec![ - "HTTP GET / DISPATCH", - "HTTP PUT / DISPATCH", - "HTTP GET / DISPATCH", - ])), - expected_max: Arc::new(StringArray::from(vec![ - "HTTP GET / DISPATCH", - "HTTP PUT / DISPATCH", - "HTTP GET / DISPATCH", - ])), - expected_null_counts: UInt64Array::from(vec![0, 0, 0]), - expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5])), - column_name: "name", - check: Check::Both, - } - .run(); - - // column "service.name" - Test { - reader: &reader, - expected_min: Arc::new(StringArray::from(vec!["frontend", "backend", "backend"])), - expected_max: Arc::new(StringArray::from(vec![ - "frontend", "frontend", "backend", - ])), - expected_null_counts: UInt64Array::from(vec![0, 0, 0]), - expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5])), - column_name: "service.name", - check: Check::Both, - } - .run(); -} - -// Boolean -#[tokio::test] -async fn test_boolean() { - // This creates a parquet files of 1 column named "bool" - // The file is created by 2 record batches each has 5 rows --> 2 row groups - let reader = TestReader { - scenario: Scenario::Boolean, - row_per_group: 5, - } - .build() - .await; - - Test { - reader: &reader, - expected_min: Arc::new(BooleanArray::from(vec![false, false])), - expected_max: Arc::new(BooleanArray::from(vec![true, false])), - expected_null_counts: UInt64Array::from(vec![1, 0]), - expected_row_counts: Some(UInt64Array::from(vec![5, 5])), - column_name: "bool", - check: Check::Both, - } - .run(); -} - -// struct array -// BUG -// https://github.com/apache/datafusion/issues/10609 -// Note that: since I have not worked on struct array before, there may be a bug in the test code rather than the real bug in the code -#[ignore] -#[tokio::test] -async fn test_struct() { - // This creates a parquet files of 1 column named "struct" - // The file is created by 1 record batch with 3 rows in the struct array - let reader = TestReader { - scenario: Scenario::StructArray, - row_per_group: 5, - } - .build() - .await; - Test { - reader: &reader, - expected_min: Arc::new(struct_array(vec![(Some(1), Some(6.0), Some(12.0))])), - expected_max: Arc::new(struct_array(vec![(Some(2), Some(8.5), Some(14.0))])), - expected_null_counts: UInt64Array::from(vec![0]), - expected_row_counts: Some(UInt64Array::from(vec![3])), - column_name: "struct", - check: Check::RowGroup, - } - .run(); -} - -// UTF8 -#[tokio::test] -async fn test_utf8() { - let reader = TestReader { - scenario: Scenario::UTF8, - row_per_group: 5, - } - .build() - .await; - - // test for utf8 - Test { - reader: &reader, - expected_min: Arc::new(StringArray::from(vec!["a", "e"])), - expected_max: Arc::new(StringArray::from(vec!["d", "i"])), - expected_null_counts: UInt64Array::from(vec![1, 0]), - expected_row_counts: Some(UInt64Array::from(vec![5, 5])), - column_name: "utf8", - check: Check::Both, - } - .run(); - - // test for large_utf8 - Test { - reader: &reader, - expected_min: Arc::new(LargeStringArray::from(vec!["a", "e"])), - expected_max: Arc::new(LargeStringArray::from(vec!["d", "i"])), - expected_null_counts: UInt64Array::from(vec![1, 0]), - expected_row_counts: Some(UInt64Array::from(vec![5, 5])), - column_name: "large_utf8", - check: Check::Both, - } - .run(); -} - -////// Files with missing statistics /////// - -#[tokio::test] -async fn test_missing_statistics() { - let reader = Int64Case { - null_values: 0, - no_null_values_start: 4, - no_null_values_end: 7, - row_per_group: 5, - enable_stats: Some(EnabledStatistics::None), - ..Default::default() - } - .build(); - - Test { - reader: &reader, - expected_min: Arc::new(Int64Array::from(vec![None])), - expected_max: Arc::new(Int64Array::from(vec![None])), - expected_null_counts: UInt64Array::from(vec![None]), - expected_row_counts: Some(UInt64Array::from(vec![3])), // still has row count statistics - column_name: "i64", - check: Check::Both, - } - .run(); -} - -/////// NEGATIVE TESTS /////// -// column not found -#[tokio::test] -async fn test_column_not_found() { - let reader = TestReader { - scenario: Scenario::Dates, - row_per_group: 5, - } - .build() - .await; - Test { - reader: &reader, - expected_min: Arc::new(Int64Array::from(vec![18262, 18565])), - expected_max: Arc::new(Int64Array::from(vec![18564, 21865])), - expected_null_counts: UInt64Array::from(vec![2, 2]), - expected_row_counts: Some(UInt64Array::from(vec![13, 7])), - column_name: "not_a_column", - check: Check::Both, - } - .run_col_not_found(); -} - -#[tokio::test] -async fn test_column_non_existent() { - // Create a schema with an additional column - // that will not have a matching parquet index - let schema = Arc::new(Schema::new(vec![ - Field::new("i8", DataType::Int8, true), - Field::new("i16", DataType::Int16, true), - Field::new("i32", DataType::Int32, true), - Field::new("i64", DataType::Int64, true), - Field::new("i_do_not_exist", DataType::Int64, true), - ])); - - let reader = TestReader { - scenario: Scenario::Int, - row_per_group: 5, - } - .build() - .await; - - Test { - reader: &reader, - // mins are [-5, -4, 0, 5] - expected_min: Arc::new(Int64Array::from(vec![None, None, None, None])), - // maxes are [-1, 0, 4, 9] - expected_max: Arc::new(Int64Array::from(vec![None, None, None, None])), - // nulls are [0, 0, 0, 0] - expected_null_counts: UInt64Array::from(vec![None, None, None, None]), - // row counts are [5, 5, 5, 5] - expected_row_counts: None, - column_name: "i_do_not_exist", - check: Check::Both, - } - .run_with_schema(&schema); -} diff --git a/datafusion/core/tests/parquet/mod.rs b/datafusion/core/tests/parquet/mod.rs index 1b68a4aa4eb3..ab19b6de527e 100644 --- a/datafusion/core/tests/parquet/mod.rs +++ b/datafusion/core/tests/parquet/mod.rs @@ -47,7 +47,6 @@ use parquet::file::properties::{EnabledStatistics, WriterProperties}; use std::sync::Arc; use tempfile::NamedTempFile; -mod arrow_statistics; mod custom_reader; // Don't run on windows as tempfiles don't seem to work the same #[cfg(not(target_os = "windows"))]