From b835f8d0dbf53cf196934acdef8e6706aee109fd Mon Sep 17 00:00:00 2001 From: Eric Fredine Date: Tue, 9 Jul 2024 07:32:55 -0700 Subject: [PATCH 01/10] Adds arrow statistics converter for parquet stastistics. --- parquet/src/arrow/arrow_reader/mod.rs | 1 + parquet/src/arrow/arrow_reader/statistics.rs | 2557 ++++++++++++++++++ 2 files changed, 2558 insertions(+) create mode 100644 parquet/src/arrow/arrow_reader/statistics.rs diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index cc369cec0ea0..0da678e2e847 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -42,6 +42,7 @@ use crate::schema::types::SchemaDescriptor; mod filter; mod selection; +mod statistics; /// Builder for constructing parquet readers into arrow. /// diff --git a/parquet/src/arrow/arrow_reader/statistics.rs b/parquet/src/arrow/arrow_reader/statistics.rs new file mode 100644 index 000000000000..8c46b7a37cd2 --- /dev/null +++ b/parquet/src/arrow/arrow_reader/statistics.rs @@ -0,0 +1,2557 @@ +// 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 crate::data_type::{ByteArray, FixedLenByteArray}; +use crate::errors::{ParquetError, Result}; +use crate::file::metadata::{ParquetColumnIndex, ParquetOffsetIndex, RowGroupMetaData}; +use crate::file::page_index::index::{Index, PageIndex}; +use crate::file::statistics::Statistics as ParquetStatistics; +use crate::schema::types::SchemaDescriptor; +use arrow_array::builder::{ + BooleanBuilder, FixedSizeBinaryBuilder, LargeStringBuilder, StringBuilder, +}; +use arrow_array::{ + new_empty_array, new_null_array, ArrayRef, 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_buffer::i256; +use arrow_schema::{DataType, Field, FieldRef, Schema, TimeUnit}; +use half::f16; +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; + /// # 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| { + arrow_err!(format!( + "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 Err(arrow_err!(format!( + "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 Err(arrow_err!(format!( + "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; + /// # 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 crate::arrow::arrow_reader::ArrowReaderBuilder; + use crate::arrow::arrow_writer::ArrowWriter; + use crate::file::metadata::{ParquetMetaData, RowGroupMetaData}; + use crate::file::properties::{EnabledStatistics, WriterProperties}; + use arrow::compute::kernels::cast_utils::Parser; + use arrow::datatypes::{i256, Date32Type, Date64Type}; + use arrow::util::test_util::parquet_test_data; + use arrow_array::{ + new_empty_array, new_null_array, Array, ArrayRef, BinaryArray, BooleanArray, Date32Array, + Date64Array, Decimal128Array, Decimal256Array, Float32Array, Float64Array, Int16Array, + Int32Array, Int64Array, Int8Array, LargeBinaryArray, RecordBatch, StringArray, StructArray, + TimestampNanosecondArray, + }; + use arrow_schema::{DataType, Field, SchemaRef}; + use bytes::Bytes; + 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) + } +} From c2e1ce29e39939bf2e2a381186f417210b32c0c0 Mon Sep 17 00:00:00 2001 From: Eric Fredine Date: Thu, 11 Jul 2024 15:02:48 -0700 Subject: [PATCH 02/10] Adds integration tests for arrow statsistics converter. --- parquet/Cargo.toml | 5 + parquet/src/arrow/arrow_reader/mod.rs | 2 +- parquet/tests/arrow_reader/mod.rs | 986 ++++++++++ parquet/tests/arrow_reader/statistics.rs | 2143 ++++++++++++++++++++++ 4 files changed, 3135 insertions(+), 1 deletion(-) create mode 100644 parquet/tests/arrow_reader/mod.rs create mode 100644 parquet/tests/arrow_reader/statistics.rs diff --git a/parquet/Cargo.toml b/parquet/Cargo.toml index 2fb808e1ec8d..553e8c518dd6 100644 --- a/parquet/Cargo.toml +++ b/parquet/Cargo.toml @@ -134,6 +134,11 @@ path = "./examples/read_with_rowgroup.rs" name = "arrow_writer_layout" required-features = ["arrow"] +[[test]] +name = "arrow_reader" +required-features = ["arrow"] +path = "./tests/arrow_reader/mod.rs" + [[bin]] name = "parquet-read" required-features = ["cli"] diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index 0da678e2e847..609908e448f4 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -42,7 +42,7 @@ use crate::schema::types::SchemaDescriptor; mod filter; mod selection; -mod statistics; +pub mod statistics; /// Builder for constructing parquet readers into arrow. /// diff --git a/parquet/tests/arrow_reader/mod.rs b/parquet/tests/arrow_reader/mod.rs new file mode 100644 index 000000000000..9a8ac89518db --- /dev/null +++ b/parquet/tests/arrow_reader/mod.rs @@ -0,0 +1,986 @@ +use arrow_array::types::{Int32Type, Int8Type}; +use arrow_array::{ + make_array, Array, ArrayRef, BinaryArray, BooleanArray, Date32Array, Date64Array, + Decimal128Array, Decimal256Array, DictionaryArray, FixedSizeBinaryArray, Float16Array, + Float32Array, Float64Array, Int16Array, Int32Array, Int64Array, Int8Array, LargeBinaryArray, + LargeStringArray, RecordBatch, StringArray, StructArray, Time32MillisecondArray, + Time32SecondArray, Time64MicrosecondArray, Time64NanosecondArray, TimestampMicrosecondArray, + TimestampMillisecondArray, TimestampNanosecondArray, TimestampSecondArray, UInt16Array, + UInt32Array, UInt64Array, UInt8Array, +}; +use arrow_buffer::i256; +use arrow_schema::{DataType, Field, Schema, TimeUnit}; +use chrono::Datelike; +use chrono::{Duration, TimeDelta}; +use half::f16; +use parquet::arrow::ArrowWriter; +use parquet::file::properties::{EnabledStatistics, WriterProperties}; +use std::sync::Arc; +use tempfile::NamedTempFile; + +mod statistics; + +// returns a struct array with columns "int32_col", "float32_col" and "float64_col" with the specified values +fn struct_array(input: Vec<(Option, Option, Option)>) -> ArrayRef { + let int_32: Int32Array = input.iter().map(|(i, _, _)| i).collect(); + let float_32: Float32Array = input.iter().map(|(_, f, _)| f).collect(); + let float_64: Float64Array = input.iter().map(|(_, _, f)| f).collect(); + + let nullable = true; + let struct_array = StructArray::from(vec![ + ( + Arc::new(Field::new("int32_col", DataType::Int32, nullable)), + Arc::new(int_32) as ArrayRef, + ), + ( + Arc::new(Field::new("float32_col", DataType::Float32, nullable)), + Arc::new(float_32) as ArrayRef, + ), + ( + Arc::new(Field::new("float64_col", DataType::Float64, nullable)), + Arc::new(float_64) as ArrayRef, + ), + ]); + Arc::new(struct_array) +} + +/// What data to use +#[derive(Debug, Clone, Copy)] +enum Scenario { + Boolean, + Timestamps, + Dates, + Int, + Int32Range, + UInt, + UInt32Range, + Time32Second, + Time32Millisecond, + Time64Nanosecond, + Time64Microsecond, + /// 7 Rows, for each i8, i16, i32, i64, u8, u16, u32, u64, f32, f64 + /// -MIN, -100, -1, 0, 1, 100, MAX + NumericLimits, + Float16, + Float32, + Float64, + Decimal, + Decimal256, + ByteArray, + Dictionary, + PeriodsInColumnNames, + StructArray, + UTF8, +} + +/// Create a test parquet file with various data types +async fn make_test_file_rg(scenario: Scenario, row_per_group: usize) -> NamedTempFile { + let mut output_file = tempfile::Builder::new() + .prefix("parquet_pruning") + .suffix(".parquet") + .tempfile() + .expect("tempfile creation"); + + let props = WriterProperties::builder() + .set_max_row_group_size(row_per_group) + .set_bloom_filter_enabled(true) + .set_statistics_enabled(EnabledStatistics::Page) + .build(); + + let batches = create_data_batch(scenario); + let schema = batches[0].schema(); + + let mut writer = ArrowWriter::try_new(&mut output_file, schema, Some(props)).unwrap(); + + for batch in batches { + writer.write(&batch).expect("writing batch"); + } + writer.close().unwrap(); + + output_file +} + +fn create_data_batch(scenario: Scenario) -> Vec { + match scenario { + Scenario::Boolean => { + vec![ + make_boolean_batch(vec![Some(true), Some(false), Some(true), Some(false), None]), + make_boolean_batch(vec![ + Some(false), + Some(false), + Some(false), + Some(false), + Some(false), + ]), + ] + } + Scenario::Timestamps => { + vec![ + make_timestamp_batch(TimeDelta::try_seconds(0).unwrap()), + make_timestamp_batch(TimeDelta::try_seconds(10).unwrap()), + make_timestamp_batch(TimeDelta::try_minutes(10).unwrap()), + make_timestamp_batch(TimeDelta::try_days(10).unwrap()), + ] + } + Scenario::Dates => { + vec![ + make_date_batch(TimeDelta::try_days(0).unwrap()), + make_date_batch(TimeDelta::try_days(10).unwrap()), + make_date_batch(TimeDelta::try_days(300).unwrap()), + make_date_batch(TimeDelta::try_days(3600).unwrap()), + ] + } + Scenario::Int => { + vec![ + make_int_batches(-5, 0), + make_int_batches(-4, 1), + make_int_batches(0, 5), + make_int_batches(5, 10), + ] + } + Scenario::Int32Range => { + vec![make_int32_range(0, 10), make_int32_range(200000, 300000)] + } + Scenario::UInt => { + vec![ + make_uint_batches(0, 5), + make_uint_batches(1, 6), + make_uint_batches(5, 10), + make_uint_batches(250, 255), + ] + } + Scenario::UInt32Range => { + vec![make_uint32_range(0, 10), make_uint32_range(200000, 300000)] + } + Scenario::NumericLimits => { + vec![make_numeric_limit_batch()] + } + Scenario::Float16 => { + vec![ + make_f16_batch( + vec![-5.0, -4.0, -3.0, -2.0, -1.0] + .into_iter() + .map(f16::from_f32) + .collect(), + ), + make_f16_batch( + vec![-4.0, -3.0, -2.0, -1.0, 0.0] + .into_iter() + .map(f16::from_f32) + .collect(), + ), + make_f16_batch( + vec![0.0, 1.0, 2.0, 3.0, 4.0] + .into_iter() + .map(f16::from_f32) + .collect(), + ), + make_f16_batch( + vec![5.0, 6.0, 7.0, 8.0, 9.0] + .into_iter() + .map(f16::from_f32) + .collect(), + ), + ] + } + Scenario::Float32 => { + vec![ + make_f32_batch(vec![-5.0, -4.0, -3.0, -2.0, -1.0]), + make_f32_batch(vec![-4.0, -3.0, -2.0, -1.0, 0.0]), + make_f32_batch(vec![0.0, 1.0, 2.0, 3.0, 4.0]), + make_f32_batch(vec![5.0, 6.0, 7.0, 8.0, 9.0]), + ] + } + Scenario::Float64 => { + vec![ + make_f64_batch(vec![-5.0, -4.0, -3.0, -2.0, -1.0]), + make_f64_batch(vec![-4.0, -3.0, -2.0, -1.0, 0.0]), + make_f64_batch(vec![0.0, 1.0, 2.0, 3.0, 4.0]), + make_f64_batch(vec![5.0, 6.0, 7.0, 8.0, 9.0]), + ] + } + Scenario::Decimal => { + // decimal record batch + vec![ + make_decimal_batch(vec![100, 200, 300, 400, 600], 9, 2), + make_decimal_batch(vec![-500, 100, 300, 400, 600], 9, 2), + make_decimal_batch(vec![2000, 3000, 3000, 4000, 6000], 9, 2), + ] + } + Scenario::Decimal256 => { + // decimal256 record batch + vec![ + make_decimal256_batch( + vec![ + i256::from(100), + i256::from(200), + i256::from(300), + i256::from(400), + i256::from(600), + ], + 9, + 2, + ), + make_decimal256_batch( + vec![ + i256::from(-500), + i256::from(100), + i256::from(300), + i256::from(400), + i256::from(600), + ], + 9, + 2, + ), + make_decimal256_batch( + vec![ + i256::from(2000), + i256::from(3000), + i256::from(3000), + i256::from(4000), + i256::from(6000), + ], + 9, + 2, + ), + ] + } + Scenario::ByteArray => { + // frontends first, then backends. All in order, except frontends 4 and 7 + // are swapped to cause a statistics false positive on the 'fixed size' column. + vec![ + make_bytearray_batch( + "all frontends", + vec![ + "frontend one", + "frontend two", + "frontend three", + "frontend seven", + "frontend five", + ], + vec![ + b"frontend one", + b"frontend two", + b"frontend three", + b"frontend seven", + b"frontend five", + ], + vec![b"fe1", b"fe2", b"fe3", b"fe7", b"fe5"], + vec![ + b"frontend one", + b"frontend two", + b"frontend three", + b"frontend seven", + b"frontend five", + ], + ), + make_bytearray_batch( + "mixed", + vec![ + "frontend six", + "frontend four", + "backend one", + "backend two", + "backend three", + ], + vec![ + b"frontend six", + b"frontend four", + b"backend one", + b"backend two", + b"backend three", + ], + vec![b"fe6", b"fe4", b"be1", b"be2", b"be3"], + vec![ + b"frontend six", + b"frontend four", + b"backend one", + b"backend two", + b"backend three", + ], + ), + make_bytearray_batch( + "all backends", + vec![ + "backend four", + "backend five", + "backend six", + "backend seven", + "backend eight", + ], + vec![ + b"backend four", + b"backend five", + b"backend six", + b"backend seven", + b"backend eight", + ], + vec![b"be4", b"be5", b"be6", b"be7", b"be8"], + vec![ + b"backend four", + b"backend five", + b"backend six", + b"backend seven", + b"backend eight", + ], + ), + ] + } + Scenario::Dictionary => { + vec![make_dict_batch()] + } + Scenario::PeriodsInColumnNames => { + vec![ + // all frontend + make_names_batch( + "HTTP GET / DISPATCH", + vec!["frontend", "frontend", "frontend", "frontend", "frontend"], + ), + // both frontend and backend + make_names_batch( + "HTTP PUT / DISPATCH", + vec!["frontend", "frontend", "backend", "backend", "backend"], + ), + // all backend + make_names_batch( + "HTTP GET / DISPATCH", + vec!["backend", "backend", "backend", "backend", "backend"], + ), + ] + } + Scenario::StructArray => { + let struct_array_data = struct_array(vec![ + (Some(1), Some(6.0), Some(12.0)), + (Some(2), Some(8.5), None), + (None, Some(8.5), Some(14.0)), + ]); + + let schema = Arc::new(Schema::new(vec![Field::new( + "struct", + struct_array_data.data_type().clone(), + true, + )])); + vec![RecordBatch::try_new(schema, vec![struct_array_data]).unwrap()] + } + Scenario::Time32Second => { + vec![ + make_time32_batches(Scenario::Time32Second, vec![18506, 18507, 18508, 18509]), + make_time32_batches(Scenario::Time32Second, vec![18510, 18511, 18512, 18513]), + make_time32_batches(Scenario::Time32Second, vec![18514, 18515, 18516, 18517]), + make_time32_batches(Scenario::Time32Second, vec![18518, 18519, 18520, 18521]), + ] + } + Scenario::Time32Millisecond => { + vec![ + make_time32_batches( + Scenario::Time32Millisecond, + vec![3600000, 3600001, 3600002, 3600003], + ), + make_time32_batches( + Scenario::Time32Millisecond, + vec![3600004, 3600005, 3600006, 3600007], + ), + make_time32_batches( + Scenario::Time32Millisecond, + vec![3600008, 3600009, 3600010, 3600011], + ), + make_time32_batches( + Scenario::Time32Millisecond, + vec![3600012, 3600013, 3600014, 3600015], + ), + ] + } + Scenario::Time64Microsecond => { + vec![ + make_time64_batches( + Scenario::Time64Microsecond, + vec![1234567890123, 1234567890124, 1234567890125, 1234567890126], + ), + make_time64_batches( + Scenario::Time64Microsecond, + vec![1234567890127, 1234567890128, 1234567890129, 1234567890130], + ), + make_time64_batches( + Scenario::Time64Microsecond, + vec![1234567890131, 1234567890132, 1234567890133, 1234567890134], + ), + make_time64_batches( + Scenario::Time64Microsecond, + vec![1234567890135, 1234567890136, 1234567890137, 1234567890138], + ), + ] + } + Scenario::Time64Nanosecond => { + vec![ + make_time64_batches( + Scenario::Time64Nanosecond, + vec![ + 987654321012345, + 987654321012346, + 987654321012347, + 987654321012348, + ], + ), + make_time64_batches( + Scenario::Time64Nanosecond, + vec![ + 987654321012349, + 987654321012350, + 987654321012351, + 987654321012352, + ], + ), + make_time64_batches( + Scenario::Time64Nanosecond, + vec![ + 987654321012353, + 987654321012354, + 987654321012355, + 987654321012356, + ], + ), + make_time64_batches( + Scenario::Time64Nanosecond, + vec![ + 987654321012357, + 987654321012358, + 987654321012359, + 987654321012360, + ], + ), + ] + } + Scenario::UTF8 => { + vec![ + make_utf8_batch(vec![Some("a"), Some("b"), Some("c"), Some("d"), None]), + make_utf8_batch(vec![Some("e"), Some("f"), Some("g"), Some("h"), Some("i")]), + ] + } + } +} + +fn make_boolean_batch(v: Vec>) -> RecordBatch { + let schema = Arc::new(Schema::new(vec![Field::new( + "bool", + DataType::Boolean, + true, + )])); + let array = Arc::new(BooleanArray::from(v)) as ArrayRef; + RecordBatch::try_new(schema, vec![array.clone()]).unwrap() +} + +/// Return record batch with a few rows of data for all of the supported timestamp types +/// values with the specified offset +/// +/// Columns are named: +/// "nanos" --> TimestampNanosecondArray +/// "nanos_timezoned" --> TimestampNanosecondArray with timezone +/// "micros" --> TimestampMicrosecondArray +/// "micros_timezoned" --> TimestampMicrosecondArray with timezone +/// "millis" --> TimestampMillisecondArray +/// "millis_timezoned" --> TimestampMillisecondArray with timezone +/// "seconds" --> TimestampSecondArray +/// "seconds_timezoned" --> TimestampSecondArray with timezone +/// "names" --> StringArray +fn make_timestamp_batch(offset: Duration) -> RecordBatch { + let ts_strings = vec![ + Some("2020-01-01T01:01:01.0000000000001"), + Some("2020-01-01T01:02:01.0000000000001"), + Some("2020-01-01T02:01:01.0000000000001"), + None, + Some("2020-01-02T01:01:01.0000000000001"), + ]; + + let tz_string = "Pacific/Efate"; + + let offset_nanos = offset.num_nanoseconds().expect("non overflow nanos"); + + let ts_nanos = ts_strings + .into_iter() + .map(|t| { + t.map(|t| { + offset_nanos + + t.parse::() + .unwrap() + .and_utc() + .timestamp_nanos_opt() + .unwrap() + }) + }) + .collect::>(); + + let ts_micros = ts_nanos + .iter() + .map(|t| t.as_ref().map(|ts_nanos| ts_nanos / 1000)) + .collect::>(); + + let ts_millis = ts_nanos + .iter() + .map(|t| t.as_ref().map(|ts_nanos| ts_nanos / 1000000)) + .collect::>(); + + let ts_seconds = ts_nanos + .iter() + .map(|t| t.as_ref().map(|ts_nanos| ts_nanos / 1000000000)) + .collect::>(); + + let names = ts_nanos + .iter() + .enumerate() + .map(|(i, _)| format!("Row {i} + {offset}")) + .collect::>(); + + let arr_nanos = TimestampNanosecondArray::from(ts_nanos.clone()); + let arr_nanos_timezoned = TimestampNanosecondArray::from(ts_nanos).with_timezone(tz_string); + let arr_micros = TimestampMicrosecondArray::from(ts_micros.clone()); + let arr_micros_timezoned = TimestampMicrosecondArray::from(ts_micros).with_timezone(tz_string); + let arr_millis = TimestampMillisecondArray::from(ts_millis.clone()); + let arr_millis_timezoned = TimestampMillisecondArray::from(ts_millis).with_timezone(tz_string); + let arr_seconds = TimestampSecondArray::from(ts_seconds.clone()); + let arr_seconds_timezoned = TimestampSecondArray::from(ts_seconds).with_timezone(tz_string); + + let names = names.iter().map(|s| s.as_str()).collect::>(); + let arr_names = StringArray::from(names); + + let schema = Schema::new(vec![ + Field::new("nanos", arr_nanos.data_type().clone(), true), + Field::new( + "nanos_timezoned", + arr_nanos_timezoned.data_type().clone(), + true, + ), + Field::new("micros", arr_micros.data_type().clone(), true), + Field::new( + "micros_timezoned", + arr_micros_timezoned.data_type().clone(), + true, + ), + Field::new("millis", arr_millis.data_type().clone(), true), + Field::new( + "millis_timezoned", + arr_millis_timezoned.data_type().clone(), + true, + ), + Field::new("seconds", arr_seconds.data_type().clone(), true), + Field::new( + "seconds_timezoned", + arr_seconds_timezoned.data_type().clone(), + true, + ), + Field::new("name", arr_names.data_type().clone(), true), + ]); + let schema = Arc::new(schema); + + RecordBatch::try_new( + schema, + vec![ + Arc::new(arr_nanos), + Arc::new(arr_nanos_timezoned), + Arc::new(arr_micros), + Arc::new(arr_micros_timezoned), + Arc::new(arr_millis), + Arc::new(arr_millis_timezoned), + Arc::new(arr_seconds), + Arc::new(arr_seconds_timezoned), + Arc::new(arr_names), + ], + ) + .unwrap() +} + +/// Return record batch with i8, i16, i32, and i64 sequences +/// +/// Columns are named +/// "i8" -> Int8Array +/// "i16" -> Int16Array +/// "i32" -> Int32Array +/// "i64" -> Int64Array +fn make_int_batches(start: i8, end: i8) -> RecordBatch { + 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), + ])); + let v8: Vec = (start..end).collect(); + let v16: Vec = (start as _..end as _).collect(); + let v32: Vec = (start as _..end as _).collect(); + let v64: Vec = (start as _..end as _).collect(); + RecordBatch::try_new( + schema, + vec![ + Arc::new(Int8Array::from(v8)) as ArrayRef, + Arc::new(Int16Array::from(v16)) as ArrayRef, + Arc::new(Int32Array::from(v32)) as ArrayRef, + Arc::new(Int64Array::from(v64)) as ArrayRef, + ], + ) + .unwrap() +} + +/// Return record batch with Time32Second, Time32Millisecond sequences +fn make_time32_batches(scenario: Scenario, v: Vec) -> RecordBatch { + match scenario { + Scenario::Time32Second => { + let schema = Arc::new(Schema::new(vec![Field::new( + "second", + DataType::Time32(TimeUnit::Second), + true, + )])); + let array = Arc::new(Time32SecondArray::from(v)) as ArrayRef; + RecordBatch::try_new(schema, vec![array]).unwrap() + } + Scenario::Time32Millisecond => { + let schema = Arc::new(Schema::new(vec![Field::new( + "millisecond", + DataType::Time32(TimeUnit::Millisecond), + true, + )])); + let array = Arc::new(Time32MillisecondArray::from(v)) as ArrayRef; + RecordBatch::try_new(schema, vec![array]).unwrap() + } + _ => panic!("Unsupported scenario for Time32"), + } +} + +/// Return record batch with Time64Microsecond, Time64Nanosecond sequences +fn make_time64_batches(scenario: Scenario, v: Vec) -> RecordBatch { + match scenario { + Scenario::Time64Microsecond => { + let schema = Arc::new(Schema::new(vec![Field::new( + "microsecond", + DataType::Time64(TimeUnit::Microsecond), + true, + )])); + let array = Arc::new(Time64MicrosecondArray::from(v)) as ArrayRef; + RecordBatch::try_new(schema, vec![array]).unwrap() + } + Scenario::Time64Nanosecond => { + let schema = Arc::new(Schema::new(vec![Field::new( + "nanosecond", + DataType::Time64(TimeUnit::Nanosecond), + true, + )])); + let array = Arc::new(Time64NanosecondArray::from(v)) as ArrayRef; + RecordBatch::try_new(schema, vec![array]).unwrap() + } + _ => panic!("Unsupported scenario for Time64"), + } +} +/// Return record batch with u8, u16, u32, and u64 sequences +/// +/// Columns are named +/// "u8" -> UInt8Array +/// "u16" -> UInt16Array +/// "u32" -> UInt32Array +/// "u64" -> UInt64Array +fn make_uint_batches(start: u8, end: u8) -> RecordBatch { + let schema = Arc::new(Schema::new(vec![ + Field::new("u8", DataType::UInt8, true), + Field::new("u16", DataType::UInt16, true), + Field::new("u32", DataType::UInt32, true), + Field::new("u64", DataType::UInt64, true), + ])); + let v8: Vec = (start..end).collect(); + let v16: Vec = (start as _..end as _).collect(); + let v32: Vec = (start as _..end as _).collect(); + let v64: Vec = (start as _..end as _).collect(); + RecordBatch::try_new( + schema, + vec![ + Arc::new(UInt8Array::from(v8)) as ArrayRef, + Arc::new(UInt16Array::from(v16)) as ArrayRef, + Arc::new(UInt32Array::from(v32)) as ArrayRef, + Arc::new(UInt64Array::from(v64)) as ArrayRef, + ], + ) + .unwrap() +} + +fn make_int32_range(start: i32, end: i32) -> RecordBatch { + let schema = Arc::new(Schema::new(vec![Field::new("i", DataType::Int32, true)])); + let v = vec![start, end]; + let array = Arc::new(Int32Array::from(v)) as ArrayRef; + RecordBatch::try_new(schema, vec![array.clone()]).unwrap() +} + +fn make_uint32_range(start: u32, end: u32) -> RecordBatch { + let schema = Arc::new(Schema::new(vec![Field::new("u", DataType::UInt32, true)])); + let v = vec![start, end]; + let array = Arc::new(UInt32Array::from(v)) as ArrayRef; + RecordBatch::try_new(schema, vec![array.clone()]).unwrap() +} + +/// Return record batch with f64 vector +/// +/// Columns are named +/// "f" -> Float64Array +fn make_f64_batch(v: Vec) -> RecordBatch { + let schema = Arc::new(Schema::new(vec![Field::new("f", DataType::Float64, true)])); + let array = Arc::new(Float64Array::from(v)) as ArrayRef; + RecordBatch::try_new(schema, vec![array.clone()]).unwrap() +} + +fn make_f32_batch(v: Vec) -> RecordBatch { + let schema = Arc::new(Schema::new(vec![Field::new("f", DataType::Float32, true)])); + let array = Arc::new(Float32Array::from(v)) as ArrayRef; + RecordBatch::try_new(schema, vec![array.clone()]).unwrap() +} + +fn make_f16_batch(v: Vec) -> RecordBatch { + let schema = Arc::new(Schema::new(vec![Field::new("f", DataType::Float16, true)])); + let array = Arc::new(Float16Array::from(v)) as ArrayRef; + RecordBatch::try_new(schema, vec![array.clone()]).unwrap() +} + +/// Return record batch with decimal vector +/// +/// Columns are named +/// "decimal_col" -> DecimalArray +fn make_decimal_batch(v: Vec, precision: u8, scale: i8) -> RecordBatch { + let schema = Arc::new(Schema::new(vec![Field::new( + "decimal_col", + DataType::Decimal128(precision, scale), + true, + )])); + let array = Arc::new( + Decimal128Array::from(v) + .with_precision_and_scale(precision, scale) + .unwrap(), + ) as ArrayRef; + RecordBatch::try_new(schema, vec![array.clone()]).unwrap() +} + +/// Return record batch with decimal256 vector +/// +/// Columns are named +/// "decimal256_col" -> Decimal256Array +fn make_decimal256_batch(v: Vec, precision: u8, scale: i8) -> RecordBatch { + let schema = Arc::new(Schema::new(vec![Field::new( + "decimal256_col", + DataType::Decimal256(precision, scale), + true, + )])); + let array = Arc::new( + Decimal256Array::from(v) + .with_precision_and_scale(precision, scale) + .unwrap(), + ) as ArrayRef; + RecordBatch::try_new(schema, vec![array]).unwrap() +} + +/// Return record batch with a few rows of data for all of the supported date +/// types with the specified offset (in days) +/// +/// Columns are named: +/// "date32" --> Date32Array +/// "date64" --> Date64Array +/// "names" --> StringArray +fn make_date_batch(offset: Duration) -> RecordBatch { + let date_strings = vec![ + Some("2020-01-01"), + Some("2020-01-02"), + Some("2020-01-03"), + None, + Some("2020-01-04"), + ]; + + let names = date_strings + .iter() + .enumerate() + .map(|(i, val)| format!("Row {i} + {offset}: {val:?}")) + .collect::>(); + + // Copied from `cast.rs` cast kernel due to lack of temporal kernels + // https://github.com/apache/arrow-rs/issues/527 + const EPOCH_DAYS_FROM_CE: i32 = 719_163; + + let date_seconds = date_strings + .iter() + .map(|t| { + t.map(|t| { + let t = t.parse::().unwrap(); + let t = t + offset; + t.num_days_from_ce() - EPOCH_DAYS_FROM_CE + }) + }) + .collect::>(); + + let date_millis = date_strings + .into_iter() + .map(|t| { + t.map(|t| { + let t = t + .parse::() + .unwrap() + .and_time(chrono::NaiveTime::from_hms_opt(0, 0, 0).unwrap()); + let t = t + offset; + t.and_utc().timestamp_millis() + }) + }) + .collect::>(); + + let arr_date32 = Date32Array::from(date_seconds); + let arr_date64 = Date64Array::from(date_millis); + + let names = names.iter().map(|s| s.as_str()).collect::>(); + let arr_names = StringArray::from(names); + + let schema = Schema::new(vec![ + Field::new("date32", arr_date32.data_type().clone(), true), + Field::new("date64", arr_date64.data_type().clone(), true), + Field::new("name", arr_names.data_type().clone(), true), + ]); + let schema = Arc::new(schema); + + RecordBatch::try_new( + schema, + vec![ + Arc::new(arr_date32), + Arc::new(arr_date64), + Arc::new(arr_names), + ], + ) + .unwrap() +} + +/// returns a batch with two columns (note "service.name" is the name +/// of the column. It is *not* a table named service.name +/// +/// name | service.name +fn make_bytearray_batch( + name: &str, + string_values: Vec<&str>, + binary_values: Vec<&[u8]>, + fixedsize_values: Vec<&[u8; 3]>, + // i64 offset. + large_binary_values: Vec<&[u8]>, +) -> RecordBatch { + let num_rows = string_values.len(); + let name: StringArray = std::iter::repeat(Some(name)).take(num_rows).collect(); + let service_string: StringArray = string_values.iter().map(Some).collect(); + let service_binary: BinaryArray = binary_values.iter().map(Some).collect(); + let service_fixedsize: FixedSizeBinaryArray = fixedsize_values + .iter() + .map(|value| Some(value.as_slice())) + .collect::>() + .into(); + let service_large_binary: LargeBinaryArray = large_binary_values.iter().map(Some).collect(); + + let schema = Schema::new(vec![ + Field::new("name", name.data_type().clone(), true), + // note the column name has a period in it! + Field::new("service_string", service_string.data_type().clone(), true), + Field::new("service_binary", service_binary.data_type().clone(), true), + Field::new( + "service_fixedsize", + service_fixedsize.data_type().clone(), + true, + ), + Field::new( + "service_large_binary", + service_large_binary.data_type().clone(), + true, + ), + ]); + let schema = Arc::new(schema); + + RecordBatch::try_new( + schema, + vec![ + Arc::new(name), + Arc::new(service_string), + Arc::new(service_binary), + Arc::new(service_fixedsize), + Arc::new(service_large_binary), + ], + ) + .unwrap() +} + +/// returns a batch with two columns (note "service.name" is the name +/// of the column. It is *not* a table named service.name +/// +/// name | service.name +fn make_names_batch(name: &str, service_name_values: Vec<&str>) -> RecordBatch { + let num_rows = service_name_values.len(); + let name: StringArray = std::iter::repeat(Some(name)).take(num_rows).collect(); + let service_name: StringArray = service_name_values.iter().map(Some).collect(); + + let schema = Schema::new(vec![ + Field::new("name", name.data_type().clone(), true), + // note the column name has a period in it! + Field::new("service.name", service_name.data_type().clone(), true), + ]); + let schema = Arc::new(schema); + + RecordBatch::try_new(schema, vec![Arc::new(name), Arc::new(service_name)]).unwrap() +} + +fn make_numeric_limit_batch() -> RecordBatch { + let i8 = Int8Array::from(vec![i8::MIN, 100, -1, 0, 1, -100, i8::MAX]); + let i16 = Int16Array::from(vec![i16::MIN, 100, -1, 0, 1, -100, i16::MAX]); + let i32 = Int32Array::from(vec![i32::MIN, 100, -1, 0, 1, -100, i32::MAX]); + let i64 = Int64Array::from(vec![i64::MIN, 100, -1, 0, 1, -100, i64::MAX]); + let u8 = UInt8Array::from(vec![u8::MIN, 100, 1, 0, 1, 100, u8::MAX]); + let u16 = UInt16Array::from(vec![u16::MIN, 100, 1, 0, 1, 100, u16::MAX]); + let u32 = UInt32Array::from(vec![u32::MIN, 100, 1, 0, 1, 100, u32::MAX]); + let u64 = UInt64Array::from(vec![u64::MIN, 100, 1, 0, 1, 100, u64::MAX]); + let f32 = Float32Array::from(vec![f32::MIN, 100.0, -1.0, 0.0, 1.0, -100.0, f32::MAX]); + let f64 = Float64Array::from(vec![f64::MIN, 100.0, -1.0, 0.0, 1.0, -100.0, f64::MAX]); + let f32_nan = Float32Array::from(vec![f32::NAN, 100.0, -1.0, 0.0, 1.0, -100.0, f32::NAN]); + let f64_nan = Float64Array::from(vec![f64::NAN, 100.0, -1.0, 0.0, 1.0, -100.0, f64::NAN]); + + RecordBatch::try_from_iter(vec![ + ("i8", Arc::new(i8) as _), + ("i16", Arc::new(i16) as _), + ("i32", Arc::new(i32) as _), + ("i64", Arc::new(i64) as _), + ("u8", Arc::new(u8) as _), + ("u16", Arc::new(u16) as _), + ("u32", Arc::new(u32) as _), + ("u64", Arc::new(u64) as _), + ("f32", Arc::new(f32) as _), + ("f64", Arc::new(f64) as _), + ("f32_nan", Arc::new(f32_nan) as _), + ("f64_nan", Arc::new(f64_nan) as _), + ]) + .unwrap() +} + +fn make_utf8_batch(value: Vec>) -> RecordBatch { + let utf8 = StringArray::from(value.clone()); + let large_utf8 = LargeStringArray::from(value); + RecordBatch::try_from_iter(vec![ + ("utf8", Arc::new(utf8) as _), + ("large_utf8", Arc::new(large_utf8) as _), + ]) + .unwrap() +} + +fn make_dict_batch() -> RecordBatch { + let values = [ + Some("abc"), + Some("def"), + None, + Some("def"), + Some("abc"), + Some("fffff"), + Some("aaa"), + ]; + let dict_i8_array = DictionaryArray::::from_iter(values.iter().cloned()); + let dict_i32_array = DictionaryArray::::from_iter(values.iter().cloned()); + + // Dictionary array of integers + let int64_values = Int64Array::from(vec![0, -100, 100]); + let keys = Int8Array::from_iter([Some(0), Some(1), None, Some(0), Some(0), Some(2), Some(0)]); + let dict_i8_int_array = + DictionaryArray::::try_new(keys, Arc::new(int64_values)).unwrap(); + + RecordBatch::try_from_iter(vec![ + ("string_dict_i8", Arc::new(dict_i8_array) as _), + ("string_dict_i32", Arc::new(dict_i32_array) as _), + ("int_dict_i8", Arc::new(dict_i8_int_array) as _), + ]) + .unwrap() +} diff --git a/parquet/tests/arrow_reader/statistics.rs b/parquet/tests/arrow_reader/statistics.rs new file mode 100644 index 000000000000..5702967ffdf4 --- /dev/null +++ b/parquet/tests/arrow_reader/statistics.rs @@ -0,0 +1,2143 @@ +// 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 super::{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 half::f16; +use parquet::arrow::arrow_reader::statistics::StatisticsConverter; +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); +} From 3103651dbe0616b2e3fe462bdcf0706ca0bbb328 Mon Sep 17 00:00:00 2001 From: Eric Fredine Date: Thu, 11 Jul 2024 15:29:29 -0700 Subject: [PATCH 03/10] Fix linting, remove todo, re-use arrow code. --- parquet/src/arrow/arrow_reader/statistics.rs | 20 +- parquet/tests/arrow_reader/mod.rs | 1305 +++++++++--------- 2 files changed, 665 insertions(+), 660 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/statistics.rs b/parquet/src/arrow/arrow_reader/statistics.rs index 8c46b7a37cd2..f20f4cfb01da 100644 --- a/parquet/src/arrow/arrow_reader/statistics.rs +++ b/parquet/src/arrow/arrow_reader/statistics.rs @@ -17,8 +17,7 @@ //! [`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 crate::arrow::buffer::bit_util::sign_extend_be; use crate::data_type::{ByteArray, FixedLenByteArray}; use crate::errors::{ParquetError, Result}; use crate::file::metadata::{ParquetColumnIndex, ParquetOffsetIndex, RowGroupMetaData}; @@ -65,19 +64,6 @@ pub(crate) fn from_bytes_to_f16(b: &[u8]) -> Option { } } -// 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` /// @@ -974,7 +960,7 @@ macro_rules! get_data_page_statistics { /// Lookups up the parquet column by name /// /// Returns the parquet column index and the corresponding arrow field -pub(crate) fn parquet_column<'a>( +pub fn parquet_column<'a>( parquet_schema: &SchemaDescriptor, arrow_schema: &'a Schema, name: &str, @@ -1129,6 +1115,7 @@ impl<'a> StatisticsConverter<'a> { /// ```no_run /// # use arrow::datatypes::Schema; /// # use arrow_array::ArrayRef; + /// # use parquet::arrow::arrow_reader::statistics::StatisticsConverter; /// # use parquet::file::metadata::ParquetMetaData; /// # fn get_parquet_metadata() -> ParquetMetaData { unimplemented!() } /// # fn get_arrow_schema() -> Schema { unimplemented!() } @@ -1235,6 +1222,7 @@ impl<'a> StatisticsConverter<'a> { /// ```no_run /// # use arrow::datatypes::Schema; /// # use arrow_array::ArrayRef; + /// # use parquet::arrow::arrow_reader::statistics::StatisticsConverter; /// # use parquet::file::metadata::ParquetMetaData; /// # fn get_parquet_metadata() -> ParquetMetaData { unimplemented!() } /// # fn get_arrow_schema() -> Schema { unimplemented!() } diff --git a/parquet/tests/arrow_reader/mod.rs b/parquet/tests/arrow_reader/mod.rs index 9a8ac89518db..4f63a505488c 100644 --- a/parquet/tests/arrow_reader/mod.rs +++ b/parquet/tests/arrow_reader/mod.rs @@ -1,8 +1,25 @@ +// 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. + use arrow_array::types::{Int32Type, Int8Type}; use arrow_array::{ - make_array, Array, ArrayRef, BinaryArray, BooleanArray, Date32Array, Date64Array, - Decimal128Array, Decimal256Array, DictionaryArray, FixedSizeBinaryArray, Float16Array, - Float32Array, Float64Array, Int16Array, Int32Array, Int64Array, Int8Array, LargeBinaryArray, + Array, ArrayRef, BinaryArray, BooleanArray, Date32Array, Date64Array, Decimal128Array, + Decimal256Array, DictionaryArray, FixedSizeBinaryArray, Float16Array, Float32Array, + Float64Array, Int16Array, Int32Array, Int64Array, Int8Array, LargeBinaryArray, LargeStringArray, RecordBatch, StringArray, StructArray, Time32MillisecondArray, Time32SecondArray, Time64MicrosecondArray, Time64NanosecondArray, TimestampMicrosecondArray, TimestampMillisecondArray, TimestampNanosecondArray, TimestampSecondArray, UInt16Array, @@ -73,596 +90,210 @@ enum Scenario { UTF8, } -/// Create a test parquet file with various data types -async fn make_test_file_rg(scenario: Scenario, row_per_group: usize) -> NamedTempFile { - let mut output_file = tempfile::Builder::new() - .prefix("parquet_pruning") - .suffix(".parquet") - .tempfile() - .expect("tempfile creation"); +fn make_boolean_batch(v: Vec>) -> RecordBatch { + let schema = Arc::new(Schema::new(vec![Field::new( + "bool", + DataType::Boolean, + true, + )])); + let array = Arc::new(BooleanArray::from(v)) as ArrayRef; + RecordBatch::try_new(schema, vec![array.clone()]).unwrap() +} - let props = WriterProperties::builder() - .set_max_row_group_size(row_per_group) - .set_bloom_filter_enabled(true) - .set_statistics_enabled(EnabledStatistics::Page) - .build(); +/// Return record batch with a few rows of data for all of the supported timestamp types +/// values with the specified offset +/// +/// Columns are named: +/// "nanos" --> TimestampNanosecondArray +/// "nanos_timezoned" --> TimestampNanosecondArray with timezone +/// "micros" --> TimestampMicrosecondArray +/// "micros_timezoned" --> TimestampMicrosecondArray with timezone +/// "millis" --> TimestampMillisecondArray +/// "millis_timezoned" --> TimestampMillisecondArray with timezone +/// "seconds" --> TimestampSecondArray +/// "seconds_timezoned" --> TimestampSecondArray with timezone +/// "names" --> StringArray +fn make_timestamp_batch(offset: Duration) -> RecordBatch { + let ts_strings = vec![ + Some("2020-01-01T01:01:01.0000000000001"), + Some("2020-01-01T01:02:01.0000000000001"), + Some("2020-01-01T02:01:01.0000000000001"), + None, + Some("2020-01-02T01:01:01.0000000000001"), + ]; - let batches = create_data_batch(scenario); - let schema = batches[0].schema(); + let tz_string = "Pacific/Efate"; - let mut writer = ArrowWriter::try_new(&mut output_file, schema, Some(props)).unwrap(); + let offset_nanos = offset.num_nanoseconds().expect("non overflow nanos"); - for batch in batches { - writer.write(&batch).expect("writing batch"); - } - writer.close().unwrap(); + let ts_nanos = ts_strings + .into_iter() + .map(|t| { + t.map(|t| { + offset_nanos + + t.parse::() + .unwrap() + .and_utc() + .timestamp_nanos_opt() + .unwrap() + }) + }) + .collect::>(); - output_file + let ts_micros = ts_nanos + .iter() + .map(|t| t.as_ref().map(|ts_nanos| ts_nanos / 1000)) + .collect::>(); + + let ts_millis = ts_nanos + .iter() + .map(|t| t.as_ref().map(|ts_nanos| ts_nanos / 1000000)) + .collect::>(); + + let ts_seconds = ts_nanos + .iter() + .map(|t| t.as_ref().map(|ts_nanos| ts_nanos / 1000000000)) + .collect::>(); + + let names = ts_nanos + .iter() + .enumerate() + .map(|(i, _)| format!("Row {i} + {offset}")) + .collect::>(); + + let arr_nanos = TimestampNanosecondArray::from(ts_nanos.clone()); + let arr_nanos_timezoned = TimestampNanosecondArray::from(ts_nanos).with_timezone(tz_string); + let arr_micros = TimestampMicrosecondArray::from(ts_micros.clone()); + let arr_micros_timezoned = TimestampMicrosecondArray::from(ts_micros).with_timezone(tz_string); + let arr_millis = TimestampMillisecondArray::from(ts_millis.clone()); + let arr_millis_timezoned = TimestampMillisecondArray::from(ts_millis).with_timezone(tz_string); + let arr_seconds = TimestampSecondArray::from(ts_seconds.clone()); + let arr_seconds_timezoned = TimestampSecondArray::from(ts_seconds).with_timezone(tz_string); + + let names = names.iter().map(|s| s.as_str()).collect::>(); + let arr_names = StringArray::from(names); + + let schema = Schema::new(vec![ + Field::new("nanos", arr_nanos.data_type().clone(), true), + Field::new( + "nanos_timezoned", + arr_nanos_timezoned.data_type().clone(), + true, + ), + Field::new("micros", arr_micros.data_type().clone(), true), + Field::new( + "micros_timezoned", + arr_micros_timezoned.data_type().clone(), + true, + ), + Field::new("millis", arr_millis.data_type().clone(), true), + Field::new( + "millis_timezoned", + arr_millis_timezoned.data_type().clone(), + true, + ), + Field::new("seconds", arr_seconds.data_type().clone(), true), + Field::new( + "seconds_timezoned", + arr_seconds_timezoned.data_type().clone(), + true, + ), + Field::new("name", arr_names.data_type().clone(), true), + ]); + let schema = Arc::new(schema); + + RecordBatch::try_new( + schema, + vec![ + Arc::new(arr_nanos), + Arc::new(arr_nanos_timezoned), + Arc::new(arr_micros), + Arc::new(arr_micros_timezoned), + Arc::new(arr_millis), + Arc::new(arr_millis_timezoned), + Arc::new(arr_seconds), + Arc::new(arr_seconds_timezoned), + Arc::new(arr_names), + ], + ) + .unwrap() } -fn create_data_batch(scenario: Scenario) -> Vec { +/// Return record batch with i8, i16, i32, and i64 sequences +/// +/// Columns are named +/// "i8" -> Int8Array +/// "i16" -> Int16Array +/// "i32" -> Int32Array +/// "i64" -> Int64Array +fn make_int_batches(start: i8, end: i8) -> RecordBatch { + 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), + ])); + let v8: Vec = (start..end).collect(); + let v16: Vec = (start as _..end as _).collect(); + let v32: Vec = (start as _..end as _).collect(); + let v64: Vec = (start as _..end as _).collect(); + RecordBatch::try_new( + schema, + vec![ + Arc::new(Int8Array::from(v8)) as ArrayRef, + Arc::new(Int16Array::from(v16)) as ArrayRef, + Arc::new(Int32Array::from(v32)) as ArrayRef, + Arc::new(Int64Array::from(v64)) as ArrayRef, + ], + ) + .unwrap() +} + +/// Return record batch with Time32Second, Time32Millisecond sequences +fn make_time32_batches(scenario: Scenario, v: Vec) -> RecordBatch { match scenario { - Scenario::Boolean => { - vec![ - make_boolean_batch(vec![Some(true), Some(false), Some(true), Some(false), None]), - make_boolean_batch(vec![ - Some(false), - Some(false), - Some(false), - Some(false), - Some(false), - ]), - ] - } - Scenario::Timestamps => { - vec![ - make_timestamp_batch(TimeDelta::try_seconds(0).unwrap()), - make_timestamp_batch(TimeDelta::try_seconds(10).unwrap()), - make_timestamp_batch(TimeDelta::try_minutes(10).unwrap()), - make_timestamp_batch(TimeDelta::try_days(10).unwrap()), - ] + Scenario::Time32Second => { + let schema = Arc::new(Schema::new(vec![Field::new( + "second", + DataType::Time32(TimeUnit::Second), + true, + )])); + let array = Arc::new(Time32SecondArray::from(v)) as ArrayRef; + RecordBatch::try_new(schema, vec![array]).unwrap() } - Scenario::Dates => { - vec![ - make_date_batch(TimeDelta::try_days(0).unwrap()), - make_date_batch(TimeDelta::try_days(10).unwrap()), - make_date_batch(TimeDelta::try_days(300).unwrap()), - make_date_batch(TimeDelta::try_days(3600).unwrap()), - ] + Scenario::Time32Millisecond => { + let schema = Arc::new(Schema::new(vec![Field::new( + "millisecond", + DataType::Time32(TimeUnit::Millisecond), + true, + )])); + let array = Arc::new(Time32MillisecondArray::from(v)) as ArrayRef; + RecordBatch::try_new(schema, vec![array]).unwrap() } - Scenario::Int => { - vec![ - make_int_batches(-5, 0), - make_int_batches(-4, 1), - make_int_batches(0, 5), - make_int_batches(5, 10), - ] + _ => panic!("Unsupported scenario for Time32"), + } +} + +/// Return record batch with Time64Microsecond, Time64Nanosecond sequences +fn make_time64_batches(scenario: Scenario, v: Vec) -> RecordBatch { + match scenario { + Scenario::Time64Microsecond => { + let schema = Arc::new(Schema::new(vec![Field::new( + "microsecond", + DataType::Time64(TimeUnit::Microsecond), + true, + )])); + let array = Arc::new(Time64MicrosecondArray::from(v)) as ArrayRef; + RecordBatch::try_new(schema, vec![array]).unwrap() } - Scenario::Int32Range => { - vec![make_int32_range(0, 10), make_int32_range(200000, 300000)] - } - Scenario::UInt => { - vec![ - make_uint_batches(0, 5), - make_uint_batches(1, 6), - make_uint_batches(5, 10), - make_uint_batches(250, 255), - ] - } - Scenario::UInt32Range => { - vec![make_uint32_range(0, 10), make_uint32_range(200000, 300000)] - } - Scenario::NumericLimits => { - vec![make_numeric_limit_batch()] - } - Scenario::Float16 => { - vec![ - make_f16_batch( - vec![-5.0, -4.0, -3.0, -2.0, -1.0] - .into_iter() - .map(f16::from_f32) - .collect(), - ), - make_f16_batch( - vec![-4.0, -3.0, -2.0, -1.0, 0.0] - .into_iter() - .map(f16::from_f32) - .collect(), - ), - make_f16_batch( - vec![0.0, 1.0, 2.0, 3.0, 4.0] - .into_iter() - .map(f16::from_f32) - .collect(), - ), - make_f16_batch( - vec![5.0, 6.0, 7.0, 8.0, 9.0] - .into_iter() - .map(f16::from_f32) - .collect(), - ), - ] - } - Scenario::Float32 => { - vec![ - make_f32_batch(vec![-5.0, -4.0, -3.0, -2.0, -1.0]), - make_f32_batch(vec![-4.0, -3.0, -2.0, -1.0, 0.0]), - make_f32_batch(vec![0.0, 1.0, 2.0, 3.0, 4.0]), - make_f32_batch(vec![5.0, 6.0, 7.0, 8.0, 9.0]), - ] - } - Scenario::Float64 => { - vec![ - make_f64_batch(vec![-5.0, -4.0, -3.0, -2.0, -1.0]), - make_f64_batch(vec![-4.0, -3.0, -2.0, -1.0, 0.0]), - make_f64_batch(vec![0.0, 1.0, 2.0, 3.0, 4.0]), - make_f64_batch(vec![5.0, 6.0, 7.0, 8.0, 9.0]), - ] - } - Scenario::Decimal => { - // decimal record batch - vec![ - make_decimal_batch(vec![100, 200, 300, 400, 600], 9, 2), - make_decimal_batch(vec![-500, 100, 300, 400, 600], 9, 2), - make_decimal_batch(vec![2000, 3000, 3000, 4000, 6000], 9, 2), - ] - } - Scenario::Decimal256 => { - // decimal256 record batch - vec![ - make_decimal256_batch( - vec![ - i256::from(100), - i256::from(200), - i256::from(300), - i256::from(400), - i256::from(600), - ], - 9, - 2, - ), - make_decimal256_batch( - vec![ - i256::from(-500), - i256::from(100), - i256::from(300), - i256::from(400), - i256::from(600), - ], - 9, - 2, - ), - make_decimal256_batch( - vec![ - i256::from(2000), - i256::from(3000), - i256::from(3000), - i256::from(4000), - i256::from(6000), - ], - 9, - 2, - ), - ] - } - Scenario::ByteArray => { - // frontends first, then backends. All in order, except frontends 4 and 7 - // are swapped to cause a statistics false positive on the 'fixed size' column. - vec![ - make_bytearray_batch( - "all frontends", - vec![ - "frontend one", - "frontend two", - "frontend three", - "frontend seven", - "frontend five", - ], - vec![ - b"frontend one", - b"frontend two", - b"frontend three", - b"frontend seven", - b"frontend five", - ], - vec![b"fe1", b"fe2", b"fe3", b"fe7", b"fe5"], - vec![ - b"frontend one", - b"frontend two", - b"frontend three", - b"frontend seven", - b"frontend five", - ], - ), - make_bytearray_batch( - "mixed", - vec![ - "frontend six", - "frontend four", - "backend one", - "backend two", - "backend three", - ], - vec![ - b"frontend six", - b"frontend four", - b"backend one", - b"backend two", - b"backend three", - ], - vec![b"fe6", b"fe4", b"be1", b"be2", b"be3"], - vec![ - b"frontend six", - b"frontend four", - b"backend one", - b"backend two", - b"backend three", - ], - ), - make_bytearray_batch( - "all backends", - vec![ - "backend four", - "backend five", - "backend six", - "backend seven", - "backend eight", - ], - vec![ - b"backend four", - b"backend five", - b"backend six", - b"backend seven", - b"backend eight", - ], - vec![b"be4", b"be5", b"be6", b"be7", b"be8"], - vec![ - b"backend four", - b"backend five", - b"backend six", - b"backend seven", - b"backend eight", - ], - ), - ] - } - Scenario::Dictionary => { - vec![make_dict_batch()] - } - Scenario::PeriodsInColumnNames => { - vec![ - // all frontend - make_names_batch( - "HTTP GET / DISPATCH", - vec!["frontend", "frontend", "frontend", "frontend", "frontend"], - ), - // both frontend and backend - make_names_batch( - "HTTP PUT / DISPATCH", - vec!["frontend", "frontend", "backend", "backend", "backend"], - ), - // all backend - make_names_batch( - "HTTP GET / DISPATCH", - vec!["backend", "backend", "backend", "backend", "backend"], - ), - ] - } - Scenario::StructArray => { - let struct_array_data = struct_array(vec![ - (Some(1), Some(6.0), Some(12.0)), - (Some(2), Some(8.5), None), - (None, Some(8.5), Some(14.0)), - ]); - - let schema = Arc::new(Schema::new(vec![Field::new( - "struct", - struct_array_data.data_type().clone(), - true, - )])); - vec![RecordBatch::try_new(schema, vec![struct_array_data]).unwrap()] - } - Scenario::Time32Second => { - vec![ - make_time32_batches(Scenario::Time32Second, vec![18506, 18507, 18508, 18509]), - make_time32_batches(Scenario::Time32Second, vec![18510, 18511, 18512, 18513]), - make_time32_batches(Scenario::Time32Second, vec![18514, 18515, 18516, 18517]), - make_time32_batches(Scenario::Time32Second, vec![18518, 18519, 18520, 18521]), - ] - } - Scenario::Time32Millisecond => { - vec![ - make_time32_batches( - Scenario::Time32Millisecond, - vec![3600000, 3600001, 3600002, 3600003], - ), - make_time32_batches( - Scenario::Time32Millisecond, - vec![3600004, 3600005, 3600006, 3600007], - ), - make_time32_batches( - Scenario::Time32Millisecond, - vec![3600008, 3600009, 3600010, 3600011], - ), - make_time32_batches( - Scenario::Time32Millisecond, - vec![3600012, 3600013, 3600014, 3600015], - ), - ] - } - Scenario::Time64Microsecond => { - vec![ - make_time64_batches( - Scenario::Time64Microsecond, - vec![1234567890123, 1234567890124, 1234567890125, 1234567890126], - ), - make_time64_batches( - Scenario::Time64Microsecond, - vec![1234567890127, 1234567890128, 1234567890129, 1234567890130], - ), - make_time64_batches( - Scenario::Time64Microsecond, - vec![1234567890131, 1234567890132, 1234567890133, 1234567890134], - ), - make_time64_batches( - Scenario::Time64Microsecond, - vec![1234567890135, 1234567890136, 1234567890137, 1234567890138], - ), - ] - } - Scenario::Time64Nanosecond => { - vec![ - make_time64_batches( - Scenario::Time64Nanosecond, - vec![ - 987654321012345, - 987654321012346, - 987654321012347, - 987654321012348, - ], - ), - make_time64_batches( - Scenario::Time64Nanosecond, - vec![ - 987654321012349, - 987654321012350, - 987654321012351, - 987654321012352, - ], - ), - make_time64_batches( - Scenario::Time64Nanosecond, - vec![ - 987654321012353, - 987654321012354, - 987654321012355, - 987654321012356, - ], - ), - make_time64_batches( - Scenario::Time64Nanosecond, - vec![ - 987654321012357, - 987654321012358, - 987654321012359, - 987654321012360, - ], - ), - ] - } - Scenario::UTF8 => { - vec![ - make_utf8_batch(vec![Some("a"), Some("b"), Some("c"), Some("d"), None]), - make_utf8_batch(vec![Some("e"), Some("f"), Some("g"), Some("h"), Some("i")]), - ] - } - } -} - -fn make_boolean_batch(v: Vec>) -> RecordBatch { - let schema = Arc::new(Schema::new(vec![Field::new( - "bool", - DataType::Boolean, - true, - )])); - let array = Arc::new(BooleanArray::from(v)) as ArrayRef; - RecordBatch::try_new(schema, vec![array.clone()]).unwrap() -} - -/// Return record batch with a few rows of data for all of the supported timestamp types -/// values with the specified offset -/// -/// Columns are named: -/// "nanos" --> TimestampNanosecondArray -/// "nanos_timezoned" --> TimestampNanosecondArray with timezone -/// "micros" --> TimestampMicrosecondArray -/// "micros_timezoned" --> TimestampMicrosecondArray with timezone -/// "millis" --> TimestampMillisecondArray -/// "millis_timezoned" --> TimestampMillisecondArray with timezone -/// "seconds" --> TimestampSecondArray -/// "seconds_timezoned" --> TimestampSecondArray with timezone -/// "names" --> StringArray -fn make_timestamp_batch(offset: Duration) -> RecordBatch { - let ts_strings = vec![ - Some("2020-01-01T01:01:01.0000000000001"), - Some("2020-01-01T01:02:01.0000000000001"), - Some("2020-01-01T02:01:01.0000000000001"), - None, - Some("2020-01-02T01:01:01.0000000000001"), - ]; - - let tz_string = "Pacific/Efate"; - - let offset_nanos = offset.num_nanoseconds().expect("non overflow nanos"); - - let ts_nanos = ts_strings - .into_iter() - .map(|t| { - t.map(|t| { - offset_nanos - + t.parse::() - .unwrap() - .and_utc() - .timestamp_nanos_opt() - .unwrap() - }) - }) - .collect::>(); - - let ts_micros = ts_nanos - .iter() - .map(|t| t.as_ref().map(|ts_nanos| ts_nanos / 1000)) - .collect::>(); - - let ts_millis = ts_nanos - .iter() - .map(|t| t.as_ref().map(|ts_nanos| ts_nanos / 1000000)) - .collect::>(); - - let ts_seconds = ts_nanos - .iter() - .map(|t| t.as_ref().map(|ts_nanos| ts_nanos / 1000000000)) - .collect::>(); - - let names = ts_nanos - .iter() - .enumerate() - .map(|(i, _)| format!("Row {i} + {offset}")) - .collect::>(); - - let arr_nanos = TimestampNanosecondArray::from(ts_nanos.clone()); - let arr_nanos_timezoned = TimestampNanosecondArray::from(ts_nanos).with_timezone(tz_string); - let arr_micros = TimestampMicrosecondArray::from(ts_micros.clone()); - let arr_micros_timezoned = TimestampMicrosecondArray::from(ts_micros).with_timezone(tz_string); - let arr_millis = TimestampMillisecondArray::from(ts_millis.clone()); - let arr_millis_timezoned = TimestampMillisecondArray::from(ts_millis).with_timezone(tz_string); - let arr_seconds = TimestampSecondArray::from(ts_seconds.clone()); - let arr_seconds_timezoned = TimestampSecondArray::from(ts_seconds).with_timezone(tz_string); - - let names = names.iter().map(|s| s.as_str()).collect::>(); - let arr_names = StringArray::from(names); - - let schema = Schema::new(vec![ - Field::new("nanos", arr_nanos.data_type().clone(), true), - Field::new( - "nanos_timezoned", - arr_nanos_timezoned.data_type().clone(), - true, - ), - Field::new("micros", arr_micros.data_type().clone(), true), - Field::new( - "micros_timezoned", - arr_micros_timezoned.data_type().clone(), - true, - ), - Field::new("millis", arr_millis.data_type().clone(), true), - Field::new( - "millis_timezoned", - arr_millis_timezoned.data_type().clone(), - true, - ), - Field::new("seconds", arr_seconds.data_type().clone(), true), - Field::new( - "seconds_timezoned", - arr_seconds_timezoned.data_type().clone(), - true, - ), - Field::new("name", arr_names.data_type().clone(), true), - ]); - let schema = Arc::new(schema); - - RecordBatch::try_new( - schema, - vec![ - Arc::new(arr_nanos), - Arc::new(arr_nanos_timezoned), - Arc::new(arr_micros), - Arc::new(arr_micros_timezoned), - Arc::new(arr_millis), - Arc::new(arr_millis_timezoned), - Arc::new(arr_seconds), - Arc::new(arr_seconds_timezoned), - Arc::new(arr_names), - ], - ) - .unwrap() -} - -/// Return record batch with i8, i16, i32, and i64 sequences -/// -/// Columns are named -/// "i8" -> Int8Array -/// "i16" -> Int16Array -/// "i32" -> Int32Array -/// "i64" -> Int64Array -fn make_int_batches(start: i8, end: i8) -> RecordBatch { - 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), - ])); - let v8: Vec = (start..end).collect(); - let v16: Vec = (start as _..end as _).collect(); - let v32: Vec = (start as _..end as _).collect(); - let v64: Vec = (start as _..end as _).collect(); - RecordBatch::try_new( - schema, - vec![ - Arc::new(Int8Array::from(v8)) as ArrayRef, - Arc::new(Int16Array::from(v16)) as ArrayRef, - Arc::new(Int32Array::from(v32)) as ArrayRef, - Arc::new(Int64Array::from(v64)) as ArrayRef, - ], - ) - .unwrap() -} - -/// Return record batch with Time32Second, Time32Millisecond sequences -fn make_time32_batches(scenario: Scenario, v: Vec) -> RecordBatch { - match scenario { - Scenario::Time32Second => { - let schema = Arc::new(Schema::new(vec![Field::new( - "second", - DataType::Time32(TimeUnit::Second), - true, - )])); - let array = Arc::new(Time32SecondArray::from(v)) as ArrayRef; - RecordBatch::try_new(schema, vec![array]).unwrap() - } - Scenario::Time32Millisecond => { - let schema = Arc::new(Schema::new(vec![Field::new( - "millisecond", - DataType::Time32(TimeUnit::Millisecond), - true, - )])); - let array = Arc::new(Time32MillisecondArray::from(v)) as ArrayRef; - RecordBatch::try_new(schema, vec![array]).unwrap() - } - _ => panic!("Unsupported scenario for Time32"), - } -} - -/// Return record batch with Time64Microsecond, Time64Nanosecond sequences -fn make_time64_batches(scenario: Scenario, v: Vec) -> RecordBatch { - match scenario { - Scenario::Time64Microsecond => { - let schema = Arc::new(Schema::new(vec![Field::new( - "microsecond", - DataType::Time64(TimeUnit::Microsecond), - true, - )])); - let array = Arc::new(Time64MicrosecondArray::from(v)) as ArrayRef; - RecordBatch::try_new(schema, vec![array]).unwrap() - } - Scenario::Time64Nanosecond => { - let schema = Arc::new(Schema::new(vec![Field::new( - "nanosecond", - DataType::Time64(TimeUnit::Nanosecond), - true, - )])); - let array = Arc::new(Time64NanosecondArray::from(v)) as ArrayRef; - RecordBatch::try_new(schema, vec![array]).unwrap() + Scenario::Time64Nanosecond => { + let schema = Arc::new(Schema::new(vec![Field::new( + "nanosecond", + DataType::Time64(TimeUnit::Nanosecond), + true, + )])); + let array = Arc::new(Time64NanosecondArray::from(v)) as ArrayRef; + RecordBatch::try_new(schema, vec![array]).unwrap() } _ => panic!("Unsupported scenario for Time64"), } @@ -914,73 +545,459 @@ fn make_names_batch(name: &str, service_name_values: Vec<&str>) -> RecordBatch { ]); let schema = Arc::new(schema); - RecordBatch::try_new(schema, vec![Arc::new(name), Arc::new(service_name)]).unwrap() + RecordBatch::try_new(schema, vec![Arc::new(name), Arc::new(service_name)]).unwrap() +} + +fn make_numeric_limit_batch() -> RecordBatch { + let i8 = Int8Array::from(vec![i8::MIN, 100, -1, 0, 1, -100, i8::MAX]); + let i16 = Int16Array::from(vec![i16::MIN, 100, -1, 0, 1, -100, i16::MAX]); + let i32 = Int32Array::from(vec![i32::MIN, 100, -1, 0, 1, -100, i32::MAX]); + let i64 = Int64Array::from(vec![i64::MIN, 100, -1, 0, 1, -100, i64::MAX]); + let u8 = UInt8Array::from(vec![u8::MIN, 100, 1, 0, 1, 100, u8::MAX]); + let u16 = UInt16Array::from(vec![u16::MIN, 100, 1, 0, 1, 100, u16::MAX]); + let u32 = UInt32Array::from(vec![u32::MIN, 100, 1, 0, 1, 100, u32::MAX]); + let u64 = UInt64Array::from(vec![u64::MIN, 100, 1, 0, 1, 100, u64::MAX]); + let f32 = Float32Array::from(vec![f32::MIN, 100.0, -1.0, 0.0, 1.0, -100.0, f32::MAX]); + let f64 = Float64Array::from(vec![f64::MIN, 100.0, -1.0, 0.0, 1.0, -100.0, f64::MAX]); + let f32_nan = Float32Array::from(vec![f32::NAN, 100.0, -1.0, 0.0, 1.0, -100.0, f32::NAN]); + let f64_nan = Float64Array::from(vec![f64::NAN, 100.0, -1.0, 0.0, 1.0, -100.0, f64::NAN]); + + RecordBatch::try_from_iter(vec![ + ("i8", Arc::new(i8) as _), + ("i16", Arc::new(i16) as _), + ("i32", Arc::new(i32) as _), + ("i64", Arc::new(i64) as _), + ("u8", Arc::new(u8) as _), + ("u16", Arc::new(u16) as _), + ("u32", Arc::new(u32) as _), + ("u64", Arc::new(u64) as _), + ("f32", Arc::new(f32) as _), + ("f64", Arc::new(f64) as _), + ("f32_nan", Arc::new(f32_nan) as _), + ("f64_nan", Arc::new(f64_nan) as _), + ]) + .unwrap() +} + +fn make_utf8_batch(value: Vec>) -> RecordBatch { + let utf8 = StringArray::from(value.clone()); + let large_utf8 = LargeStringArray::from(value); + RecordBatch::try_from_iter(vec![ + ("utf8", Arc::new(utf8) as _), + ("large_utf8", Arc::new(large_utf8) as _), + ]) + .unwrap() +} + +fn make_dict_batch() -> RecordBatch { + let values = [ + Some("abc"), + Some("def"), + None, + Some("def"), + Some("abc"), + Some("fffff"), + Some("aaa"), + ]; + let dict_i8_array = DictionaryArray::::from_iter(values.iter().cloned()); + let dict_i32_array = DictionaryArray::::from_iter(values.iter().cloned()); + + // Dictionary array of integers + let int64_values = Int64Array::from(vec![0, -100, 100]); + let keys = Int8Array::from_iter([Some(0), Some(1), None, Some(0), Some(0), Some(2), Some(0)]); + let dict_i8_int_array = + DictionaryArray::::try_new(keys, Arc::new(int64_values)).unwrap(); + + RecordBatch::try_from_iter(vec![ + ("string_dict_i8", Arc::new(dict_i8_array) as _), + ("string_dict_i32", Arc::new(dict_i32_array) as _), + ("int_dict_i8", Arc::new(dict_i8_int_array) as _), + ]) + .unwrap() +} + +fn create_data_batch(scenario: Scenario) -> Vec { + match scenario { + Scenario::Boolean => { + vec![ + make_boolean_batch(vec![Some(true), Some(false), Some(true), Some(false), None]), + make_boolean_batch(vec![ + Some(false), + Some(false), + Some(false), + Some(false), + Some(false), + ]), + ] + } + Scenario::Timestamps => { + vec![ + make_timestamp_batch(TimeDelta::try_seconds(0).unwrap()), + make_timestamp_batch(TimeDelta::try_seconds(10).unwrap()), + make_timestamp_batch(TimeDelta::try_minutes(10).unwrap()), + make_timestamp_batch(TimeDelta::try_days(10).unwrap()), + ] + } + Scenario::Dates => { + vec![ + make_date_batch(TimeDelta::try_days(0).unwrap()), + make_date_batch(TimeDelta::try_days(10).unwrap()), + make_date_batch(TimeDelta::try_days(300).unwrap()), + make_date_batch(TimeDelta::try_days(3600).unwrap()), + ] + } + Scenario::Int => { + vec![ + make_int_batches(-5, 0), + make_int_batches(-4, 1), + make_int_batches(0, 5), + make_int_batches(5, 10), + ] + } + Scenario::Int32Range => { + vec![make_int32_range(0, 10), make_int32_range(200000, 300000)] + } + Scenario::UInt => { + vec![ + make_uint_batches(0, 5), + make_uint_batches(1, 6), + make_uint_batches(5, 10), + make_uint_batches(250, 255), + ] + } + Scenario::UInt32Range => { + vec![make_uint32_range(0, 10), make_uint32_range(200000, 300000)] + } + Scenario::NumericLimits => { + vec![make_numeric_limit_batch()] + } + Scenario::Float16 => { + vec![ + make_f16_batch( + vec![-5.0, -4.0, -3.0, -2.0, -1.0] + .into_iter() + .map(f16::from_f32) + .collect(), + ), + make_f16_batch( + vec![-4.0, -3.0, -2.0, -1.0, 0.0] + .into_iter() + .map(f16::from_f32) + .collect(), + ), + make_f16_batch( + vec![0.0, 1.0, 2.0, 3.0, 4.0] + .into_iter() + .map(f16::from_f32) + .collect(), + ), + make_f16_batch( + vec![5.0, 6.0, 7.0, 8.0, 9.0] + .into_iter() + .map(f16::from_f32) + .collect(), + ), + ] + } + Scenario::Float32 => { + vec![ + make_f32_batch(vec![-5.0, -4.0, -3.0, -2.0, -1.0]), + make_f32_batch(vec![-4.0, -3.0, -2.0, -1.0, 0.0]), + make_f32_batch(vec![0.0, 1.0, 2.0, 3.0, 4.0]), + make_f32_batch(vec![5.0, 6.0, 7.0, 8.0, 9.0]), + ] + } + Scenario::Float64 => { + vec![ + make_f64_batch(vec![-5.0, -4.0, -3.0, -2.0, -1.0]), + make_f64_batch(vec![-4.0, -3.0, -2.0, -1.0, 0.0]), + make_f64_batch(vec![0.0, 1.0, 2.0, 3.0, 4.0]), + make_f64_batch(vec![5.0, 6.0, 7.0, 8.0, 9.0]), + ] + } + Scenario::Decimal => { + // decimal record batch + vec![ + make_decimal_batch(vec![100, 200, 300, 400, 600], 9, 2), + make_decimal_batch(vec![-500, 100, 300, 400, 600], 9, 2), + make_decimal_batch(vec![2000, 3000, 3000, 4000, 6000], 9, 2), + ] + } + Scenario::Decimal256 => { + // decimal256 record batch + vec![ + make_decimal256_batch( + vec![ + i256::from(100), + i256::from(200), + i256::from(300), + i256::from(400), + i256::from(600), + ], + 9, + 2, + ), + make_decimal256_batch( + vec![ + i256::from(-500), + i256::from(100), + i256::from(300), + i256::from(400), + i256::from(600), + ], + 9, + 2, + ), + make_decimal256_batch( + vec![ + i256::from(2000), + i256::from(3000), + i256::from(3000), + i256::from(4000), + i256::from(6000), + ], + 9, + 2, + ), + ] + } + Scenario::ByteArray => { + // frontends first, then backends. All in order, except frontends 4 and 7 + // are swapped to cause a statistics false positive on the 'fixed size' column. + vec![ + make_bytearray_batch( + "all frontends", + vec![ + "frontend one", + "frontend two", + "frontend three", + "frontend seven", + "frontend five", + ], + vec![ + b"frontend one", + b"frontend two", + b"frontend three", + b"frontend seven", + b"frontend five", + ], + vec![b"fe1", b"fe2", b"fe3", b"fe7", b"fe5"], + vec![ + b"frontend one", + b"frontend two", + b"frontend three", + b"frontend seven", + b"frontend five", + ], + ), + make_bytearray_batch( + "mixed", + vec![ + "frontend six", + "frontend four", + "backend one", + "backend two", + "backend three", + ], + vec![ + b"frontend six", + b"frontend four", + b"backend one", + b"backend two", + b"backend three", + ], + vec![b"fe6", b"fe4", b"be1", b"be2", b"be3"], + vec![ + b"frontend six", + b"frontend four", + b"backend one", + b"backend two", + b"backend three", + ], + ), + make_bytearray_batch( + "all backends", + vec![ + "backend four", + "backend five", + "backend six", + "backend seven", + "backend eight", + ], + vec![ + b"backend four", + b"backend five", + b"backend six", + b"backend seven", + b"backend eight", + ], + vec![b"be4", b"be5", b"be6", b"be7", b"be8"], + vec![ + b"backend four", + b"backend five", + b"backend six", + b"backend seven", + b"backend eight", + ], + ), + ] + } + Scenario::Dictionary => { + vec![make_dict_batch()] + } + Scenario::PeriodsInColumnNames => { + vec![ + // all frontend + make_names_batch( + "HTTP GET / DISPATCH", + vec!["frontend", "frontend", "frontend", "frontend", "frontend"], + ), + // both frontend and backend + make_names_batch( + "HTTP PUT / DISPATCH", + vec!["frontend", "frontend", "backend", "backend", "backend"], + ), + // all backend + make_names_batch( + "HTTP GET / DISPATCH", + vec!["backend", "backend", "backend", "backend", "backend"], + ), + ] + } + Scenario::StructArray => { + let struct_array_data = struct_array(vec![ + (Some(1), Some(6.0), Some(12.0)), + (Some(2), Some(8.5), None), + (None, Some(8.5), Some(14.0)), + ]); + + let schema = Arc::new(Schema::new(vec![Field::new( + "struct", + struct_array_data.data_type().clone(), + true, + )])); + vec![RecordBatch::try_new(schema, vec![struct_array_data]).unwrap()] + } + Scenario::Time32Second => { + vec![ + make_time32_batches(Scenario::Time32Second, vec![18506, 18507, 18508, 18509]), + make_time32_batches(Scenario::Time32Second, vec![18510, 18511, 18512, 18513]), + make_time32_batches(Scenario::Time32Second, vec![18514, 18515, 18516, 18517]), + make_time32_batches(Scenario::Time32Second, vec![18518, 18519, 18520, 18521]), + ] + } + Scenario::Time32Millisecond => { + vec![ + make_time32_batches( + Scenario::Time32Millisecond, + vec![3600000, 3600001, 3600002, 3600003], + ), + make_time32_batches( + Scenario::Time32Millisecond, + vec![3600004, 3600005, 3600006, 3600007], + ), + make_time32_batches( + Scenario::Time32Millisecond, + vec![3600008, 3600009, 3600010, 3600011], + ), + make_time32_batches( + Scenario::Time32Millisecond, + vec![3600012, 3600013, 3600014, 3600015], + ), + ] + } + Scenario::Time64Microsecond => { + vec![ + make_time64_batches( + Scenario::Time64Microsecond, + vec![1234567890123, 1234567890124, 1234567890125, 1234567890126], + ), + make_time64_batches( + Scenario::Time64Microsecond, + vec![1234567890127, 1234567890128, 1234567890129, 1234567890130], + ), + make_time64_batches( + Scenario::Time64Microsecond, + vec![1234567890131, 1234567890132, 1234567890133, 1234567890134], + ), + make_time64_batches( + Scenario::Time64Microsecond, + vec![1234567890135, 1234567890136, 1234567890137, 1234567890138], + ), + ] + } + Scenario::Time64Nanosecond => { + vec![ + make_time64_batches( + Scenario::Time64Nanosecond, + vec![ + 987654321012345, + 987654321012346, + 987654321012347, + 987654321012348, + ], + ), + make_time64_batches( + Scenario::Time64Nanosecond, + vec![ + 987654321012349, + 987654321012350, + 987654321012351, + 987654321012352, + ], + ), + make_time64_batches( + Scenario::Time64Nanosecond, + vec![ + 987654321012353, + 987654321012354, + 987654321012355, + 987654321012356, + ], + ), + make_time64_batches( + Scenario::Time64Nanosecond, + vec![ + 987654321012357, + 987654321012358, + 987654321012359, + 987654321012360, + ], + ), + ] + } + Scenario::UTF8 => { + vec![ + make_utf8_batch(vec![Some("a"), Some("b"), Some("c"), Some("d"), None]), + make_utf8_batch(vec![Some("e"), Some("f"), Some("g"), Some("h"), Some("i")]), + ] + } + } } -fn make_numeric_limit_batch() -> RecordBatch { - let i8 = Int8Array::from(vec![i8::MIN, 100, -1, 0, 1, -100, i8::MAX]); - let i16 = Int16Array::from(vec![i16::MIN, 100, -1, 0, 1, -100, i16::MAX]); - let i32 = Int32Array::from(vec![i32::MIN, 100, -1, 0, 1, -100, i32::MAX]); - let i64 = Int64Array::from(vec![i64::MIN, 100, -1, 0, 1, -100, i64::MAX]); - let u8 = UInt8Array::from(vec![u8::MIN, 100, 1, 0, 1, 100, u8::MAX]); - let u16 = UInt16Array::from(vec![u16::MIN, 100, 1, 0, 1, 100, u16::MAX]); - let u32 = UInt32Array::from(vec![u32::MIN, 100, 1, 0, 1, 100, u32::MAX]); - let u64 = UInt64Array::from(vec![u64::MIN, 100, 1, 0, 1, 100, u64::MAX]); - let f32 = Float32Array::from(vec![f32::MIN, 100.0, -1.0, 0.0, 1.0, -100.0, f32::MAX]); - let f64 = Float64Array::from(vec![f64::MIN, 100.0, -1.0, 0.0, 1.0, -100.0, f64::MAX]); - let f32_nan = Float32Array::from(vec![f32::NAN, 100.0, -1.0, 0.0, 1.0, -100.0, f32::NAN]); - let f64_nan = Float64Array::from(vec![f64::NAN, 100.0, -1.0, 0.0, 1.0, -100.0, f64::NAN]); +/// Create a test parquet file with various data types +async fn make_test_file_rg(scenario: Scenario, row_per_group: usize) -> NamedTempFile { + let mut output_file = tempfile::Builder::new() + .prefix("parquet_pruning") + .suffix(".parquet") + .tempfile() + .expect("tempfile creation"); - RecordBatch::try_from_iter(vec![ - ("i8", Arc::new(i8) as _), - ("i16", Arc::new(i16) as _), - ("i32", Arc::new(i32) as _), - ("i64", Arc::new(i64) as _), - ("u8", Arc::new(u8) as _), - ("u16", Arc::new(u16) as _), - ("u32", Arc::new(u32) as _), - ("u64", Arc::new(u64) as _), - ("f32", Arc::new(f32) as _), - ("f64", Arc::new(f64) as _), - ("f32_nan", Arc::new(f32_nan) as _), - ("f64_nan", Arc::new(f64_nan) as _), - ]) - .unwrap() -} + let props = WriterProperties::builder() + .set_max_row_group_size(row_per_group) + .set_bloom_filter_enabled(true) + .set_statistics_enabled(EnabledStatistics::Page) + .build(); -fn make_utf8_batch(value: Vec>) -> RecordBatch { - let utf8 = StringArray::from(value.clone()); - let large_utf8 = LargeStringArray::from(value); - RecordBatch::try_from_iter(vec![ - ("utf8", Arc::new(utf8) as _), - ("large_utf8", Arc::new(large_utf8) as _), - ]) - .unwrap() -} + let batches = create_data_batch(scenario); + let schema = batches[0].schema(); -fn make_dict_batch() -> RecordBatch { - let values = [ - Some("abc"), - Some("def"), - None, - Some("def"), - Some("abc"), - Some("fffff"), - Some("aaa"), - ]; - let dict_i8_array = DictionaryArray::::from_iter(values.iter().cloned()); - let dict_i32_array = DictionaryArray::::from_iter(values.iter().cloned()); + let mut writer = ArrowWriter::try_new(&mut output_file, schema, Some(props)).unwrap(); - // Dictionary array of integers - let int64_values = Int64Array::from(vec![0, -100, 100]); - let keys = Int8Array::from_iter([Some(0), Some(1), None, Some(0), Some(0), Some(2), Some(0)]); - let dict_i8_int_array = - DictionaryArray::::try_new(keys, Arc::new(int64_values)).unwrap(); + for batch in batches { + writer.write(&batch).expect("writing batch"); + } + writer.close().unwrap(); - RecordBatch::try_from_iter(vec![ - ("string_dict_i8", Arc::new(dict_i8_array) as _), - ("string_dict_i32", Arc::new(dict_i32_array) as _), - ("int_dict_i8", Arc::new(dict_i8_int_array) as _), - ]) - .unwrap() + output_file } From 1a2a893059b3d2d92f94d4e3c3ddd7b4cafde8fe Mon Sep 17 00:00:00 2001 From: Eric Fredine Date: Thu, 11 Jul 2024 16:02:51 -0700 Subject: [PATCH 04/10] Remove commented out debug::log statements. --- parquet/src/arrow/arrow_reader/statistics.rs | 9 --------- 1 file changed, 9 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/statistics.rs b/parquet/src/arrow/arrow_reader/statistics.rs index f20f4cfb01da..01745a9c80fe 100644 --- a/parquet/src/arrow/arrow_reader/statistics.rs +++ b/parquet/src/arrow/arrow_reader/statistics.rs @@ -386,7 +386,6 @@ macro_rules! get_statistics { }; let Ok(x) = std::str::from_utf8(x) else { - // log::debug!("Utf8 statistics is a non-UTF8 value, ignoring it."); builder.append_null(); continue; }; @@ -405,7 +404,6 @@ macro_rules! get_statistics { }; let Ok(x) = std::str::from_utf8(x) else { - // log::debug!("Utf8 statistics is a non-UTF8 value, ignoring it."); builder.append_null(); continue; }; @@ -425,11 +423,6 @@ macro_rules! get_statistics { // 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; } @@ -838,7 +831,6 @@ macro_rules! get_data_page_statistics { }; 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; }; @@ -859,7 +851,6 @@ macro_rules! get_data_page_statistics { }; 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; }; From fa5bd31109a6cf7c1fa8af61c7d869b5e9585349 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 15 Jul 2024 16:08:03 -0400 Subject: [PATCH 05/10] Move parquet_column to lib.rs --- parquet/src/arrow/arrow_reader/statistics.rs | 29 ++------------------ parquet/src/arrow/mod.rs | 26 ++++++++++++++++++ 2 files changed, 28 insertions(+), 27 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/statistics.rs b/parquet/src/arrow/arrow_reader/statistics.rs index 01745a9c80fe..3c5a7b69fc6d 100644 --- a/parquet/src/arrow/arrow_reader/statistics.rs +++ b/parquet/src/arrow/arrow_reader/statistics.rs @@ -18,6 +18,7 @@ //! [`StatisticsConverter`] to convert statistics in parquet format to arrow [`ArrayRef`]. use crate::arrow::buffer::bit_util::sign_extend_be; +use crate::arrow::parquet_column; use crate::data_type::{ByteArray, FixedLenByteArray}; use crate::errors::{ParquetError, Result}; use crate::file::metadata::{ParquetColumnIndex, ParquetOffsetIndex, RowGroupMetaData}; @@ -36,7 +37,7 @@ use arrow_array::{ UInt32Array, UInt64Array, UInt8Array, }; use arrow_buffer::i256; -use arrow_schema::{DataType, Field, FieldRef, Schema, TimeUnit}; +use arrow_schema::{DataType, Field, Schema, TimeUnit}; use half::f16; use paste::paste; use std::sync::Arc; @@ -947,32 +948,6 @@ macro_rules! get_data_page_statistics { } } } - -/// Lookups up the parquet column by name -/// -/// Returns the parquet column index and the corresponding arrow field -pub 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`] /// diff --git a/parquet/src/arrow/mod.rs b/parquet/src/arrow/mod.rs index 950226aef721..5c36891434c3 100644 --- a/parquet/src/arrow/mod.rs +++ b/parquet/src/arrow/mod.rs @@ -114,6 +114,7 @@ pub use self::async_reader::ParquetRecordBatchStreamBuilder; #[cfg(feature = "async")] pub use self::async_writer::AsyncArrowWriter; use crate::schema::types::SchemaDescriptor; +use arrow_schema::{FieldRef, Schema}; pub use self::schema::{ arrow_to_parquet_schema, parquet_to_arrow_field_levels, parquet_to_arrow_schema, @@ -210,3 +211,28 @@ impl ProjectionMask { self.mask.as_ref().map(|m| m[leaf_idx]).unwrap_or(true) } } + +/// Lookups up the parquet column by name +/// +/// Returns the parquet column index and the corresponding arrow field +pub 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)) +} From 1a0f23bae22f8a5df7fbd06fc279a7fdf5c6d650 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 15 Jul 2024 16:11:02 -0400 Subject: [PATCH 06/10] doc tweaks --- parquet/src/arrow/arrow_reader/statistics.rs | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/statistics.rs b/parquet/src/arrow/arrow_reader/statistics.rs index 3c5a7b69fc6d..d1bab3e8720f 100644 --- a/parquet/src/arrow/arrow_reader/statistics.rs +++ b/parquet/src/arrow/arrow_reader/statistics.rs @@ -1366,9 +1366,7 @@ impl<'a> StatisticsConverter<'a> { max_page_statistics(Some(data_type), iter) } - /// Extract the null counts from Data Page statistics. - /// - /// The returned Array is an [`UInt64Array`] + /// Returns a [`UInt64Array`] with null counts for each data page. /// /// See docs on [`Self::data_page_mins`] for details. pub fn data_page_null_counts( @@ -1397,7 +1395,7 @@ impl<'a> StatisticsConverter<'a> { null_counts_page_statistics(iter) } - /// Returns an [`ArrayRef`] with row counts for each row group. + /// Returns a [`UInt64Array`] with row counts for each data page. /// /// This function iterates over the given row group indexes and computes /// the row count for each page in the specified column. From 3438746de0474510ba333a72fe3b12c95627c39b Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 15 Jul 2024 16:40:49 -0400 Subject: [PATCH 07/10] Add benchmark --- parquet/Cargo.toml | 7 +- parquet/benches/arrow_statistics.rs | 269 ++++++++++++++++++++++++++++ 2 files changed, 275 insertions(+), 1 deletion(-) create mode 100644 parquet/benches/arrow_statistics.rs diff --git a/parquet/Cargo.toml b/parquet/Cargo.toml index 553e8c518dd6..2cc12a81dea5 100644 --- a/parquet/Cargo.toml +++ b/parquet/Cargo.toml @@ -185,6 +185,12 @@ name = "arrow_reader" required-features = ["arrow", "test_common", "experimental"] harness = false +[[bench]] +name = "arrow_statistics" +required-features = ["arrow"] +harness = false + + [[bench]] name = "compression" required-features = ["experimental", "default"] @@ -195,7 +201,6 @@ name = "encoding" required-features = ["experimental", "default"] harness = false - [[bench]] name = "metadata" harness = false diff --git a/parquet/benches/arrow_statistics.rs b/parquet/benches/arrow_statistics.rs new file mode 100644 index 000000000000..ebc2fb38a7ec --- /dev/null +++ b/parquet/benches/arrow_statistics.rs @@ -0,0 +1,269 @@ +// 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 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 parquet::arrow::arrow_reader::statistics::StatisticsConverter; +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); From 5c8c1bae4d70962cbed58e0be4ff2d519fd4d9fb Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 16 Jul 2024 09:52:24 -0400 Subject: [PATCH 08/10] Add parquet_column_index and arrow_field accessors + test --- parquet/src/arrow/arrow_reader/statistics.rs | 40 +++++++++++++++----- 1 file changed, 30 insertions(+), 10 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/statistics.rs b/parquet/src/arrow/arrow_reader/statistics.rs index d1bab3e8720f..d6f1bb22e10f 100644 --- a/parquet/src/arrow/arrow_reader/statistics.rs +++ b/parquet/src/arrow/arrow_reader/statistics.rs @@ -1064,12 +1064,25 @@ where #[derive(Debug)] pub struct StatisticsConverter<'a> { /// the index of the matched column in the parquet schema - parquet_index: Option, + parquet_column_index: Option, /// The field (with data type) of the column in the arrow schema arrow_field: &'a Field, } impl<'a> StatisticsConverter<'a> { + /// Return the index of the column in the parquet schema, if any + /// + /// Returns `None` if the column is was present in the Arrow schema, but not + /// present in the parquet file + pub fn parquet_column_index(&self) -> Option { + self.parquet_column_index + } + + /// Return the arrow schema's [`Field]` of the column in the arrow schema + pub fn arrow_field(&self) -> &'a Field { + self.arrow_field + } + /// Returns a [`UInt64Array`] with row counts for each row group /// /// # Return Value @@ -1102,7 +1115,7 @@ impl<'a> StatisticsConverter<'a> { where I: IntoIterator, { - let Some(_) = self.parquet_index else { + let Some(_) = self.parquet_column_index else { return Ok(None); }; @@ -1159,7 +1172,7 @@ impl<'a> StatisticsConverter<'a> { }; Ok(Self { - parquet_index, + parquet_column_index: parquet_index, arrow_field, }) } @@ -1210,7 +1223,7 @@ impl<'a> StatisticsConverter<'a> { { let data_type = self.arrow_field.data_type(); - let Some(parquet_index) = self.parquet_index else { + let Some(parquet_index) = self.parquet_column_index else { return Ok(self.make_null_array(data_type, metadatas)); }; @@ -1229,7 +1242,7 @@ impl<'a> StatisticsConverter<'a> { { let data_type = self.arrow_field.data_type(); - let Some(parquet_index) = self.parquet_index else { + let Some(parquet_index) = self.parquet_column_index else { return Ok(self.make_null_array(data_type, metadatas)); }; @@ -1246,7 +1259,7 @@ impl<'a> StatisticsConverter<'a> { where I: IntoIterator, { - let Some(parquet_index) = self.parquet_index else { + let Some(parquet_index) = self.parquet_column_index else { let num_row_groups = metadatas.into_iter().count(); return Ok(UInt64Array::from_iter( std::iter::repeat(None).take(num_row_groups), @@ -1322,7 +1335,7 @@ impl<'a> StatisticsConverter<'a> { { let data_type = self.arrow_field.data_type(); - let Some(parquet_index) = self.parquet_index else { + let Some(parquet_index) = self.parquet_column_index else { return Ok(self.make_null_array(data_type, row_group_indices)); }; @@ -1351,7 +1364,7 @@ impl<'a> StatisticsConverter<'a> { { let data_type = self.arrow_field.data_type(); - let Some(parquet_index) = self.parquet_index else { + let Some(parquet_index) = self.parquet_column_index else { return Ok(self.make_null_array(data_type, row_group_indices)); }; @@ -1378,7 +1391,7 @@ impl<'a> StatisticsConverter<'a> { where I: IntoIterator, { - let Some(parquet_index) = self.parquet_index else { + let Some(parquet_index) = self.parquet_column_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), @@ -1421,7 +1434,7 @@ impl<'a> StatisticsConverter<'a> { where I: IntoIterator, { - let Some(parquet_index) = self.parquet_index else { + let Some(parquet_index) = self.parquet_column_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. @@ -2366,6 +2379,13 @@ mod test { let converter = StatisticsConverter::try_new(name, arrow_schema, parquet_schema).unwrap(); + + // test accessors on the converter + let parquet_column_index = + parquet_column(parquet_schema, &arrow_schema, name).map(|(idx, _field)| idx); + assert_eq!(converter.parquet_column_index(), parquet_column_index); + assert_eq!(converter.arrow_field().name(), name); + let actual_min = converter.row_group_mins(row_groups.iter()).unwrap(); assert_eq!(&expected_min, &actual_min, "column {name}"); From f993b08f9bccbedcd725ae7f29a4b7c8de09f412 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 16 Jul 2024 10:00:56 -0400 Subject: [PATCH 09/10] Copy edit docs obsessively --- parquet/src/arrow/arrow_reader/statistics.rs | 39 ++++++++++++-------- 1 file changed, 23 insertions(+), 16 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/statistics.rs b/parquet/src/arrow/arrow_reader/statistics.rs index d6f1bb22e10f..9a6c947452e9 100644 --- a/parquet/src/arrow/arrow_reader/statistics.rs +++ b/parquet/src/arrow/arrow_reader/statistics.rs @@ -1046,31 +1046,33 @@ where /// 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 +/// This is used to convert Parquet statistics to Arrow [`ArrayRef`], with +/// proper type conversions. This information can be used for pruning Parquet +/// files, row groups, and data pages based on the statistics embedded in +/// Parquet metadata. /// /// # 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 converter ues the schema of the Parquet file and the Arrow schema 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 and this structure handles mapping them to the `i128` +/// representation used in Arrow. /// -/// The parquet_schema and arrow_schema do not have to be identical (for +/// Note: 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. +/// match the column in the Parquet schema to the column in the Arrow schema. #[derive(Debug)] pub struct StatisticsConverter<'a> { - /// the index of the matched column in the parquet schema + /// the index of the matched column in the Parquet schema parquet_column_index: Option, - /// The field (with data type) of the column in the arrow schema + /// The field (with data type) of the column in the Arrow schema arrow_field: &'a Field, } impl<'a> StatisticsConverter<'a> { - /// Return the index of the column in the parquet schema, if any + /// Return the index of the column in the Parquet schema, if any /// /// Returns `None` if the column is was present in the Arrow schema, but not /// present in the parquet file @@ -1078,7 +1080,7 @@ impl<'a> StatisticsConverter<'a> { self.parquet_column_index } - /// Return the arrow schema's [`Field]` of the column in the arrow schema + /// Return the arrow schema's [`Field]` of the column in the Arrow schema pub fn arrow_field(&self) -> &'a Field { self.arrow_field } @@ -1093,7 +1095,7 @@ impl<'a> StatisticsConverter<'a> { /// # Example /// ```no_run /// # use arrow::datatypes::Schema; - /// # use arrow_array::ArrayRef; + /// # use arrow_array::{ArrayRef, UInt64Array}; /// # use parquet::arrow::arrow_reader::statistics::StatisticsConverter; /// # use parquet::file::metadata::ParquetMetaData; /// # fn get_parquet_metadata() -> ParquetMetaData { unimplemented!() } @@ -1109,7 +1111,9 @@ impl<'a> StatisticsConverter<'a> { /// let row_counts = converter.row_group_row_counts(metadata /// .row_groups() /// .iter() - /// ); + /// ).unwrap(); + /// // file had 2 row groups, with 1024 and 23 rows respectively + /// assert_eq!(row_counts, Some(UInt64Array::from(vec![1024, 23]))); /// ``` pub fn row_group_row_counts(&self, metadatas: I) -> Result> where @@ -1199,8 +1203,9 @@ impl<'a> StatisticsConverter<'a> { /// /// # Example /// ```no_run + /// # use std::sync::Arc; /// # use arrow::datatypes::Schema; - /// # use arrow_array::ArrayRef; + /// # use arrow_array::{ArrayRef, Float64Array}; /// # use parquet::arrow::arrow_reader::statistics::StatisticsConverter; /// # use parquet::file::metadata::ParquetMetaData; /// # fn get_parquet_metadata() -> ParquetMetaData { unimplemented!() } @@ -1216,6 +1221,8 @@ impl<'a> StatisticsConverter<'a> { /// let min_values: ArrayRef = converter /// .row_group_mins(metadata.row_groups().iter()) /// .unwrap(); + /// // if "foo" is a Float64 value, the returned array will contain Float64 values + /// assert_eq!(min_values, Arc::new(Float64Array::from(vec![Some(1.0), Some(2.0)])) as _); /// ``` pub fn row_group_mins(&self, metadatas: I) -> Result where From 469fed9777866d459a6f62aab1b49a8d13508004 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 16 Jul 2024 10:05:29 -0400 Subject: [PATCH 10/10] clippy --- parquet/src/arrow/arrow_reader/statistics.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/parquet/src/arrow/arrow_reader/statistics.rs b/parquet/src/arrow/arrow_reader/statistics.rs index 9a6c947452e9..d536792b827b 100644 --- a/parquet/src/arrow/arrow_reader/statistics.rs +++ b/parquet/src/arrow/arrow_reader/statistics.rs @@ -2389,7 +2389,7 @@ mod test { // test accessors on the converter let parquet_column_index = - parquet_column(parquet_schema, &arrow_schema, name).map(|(idx, _field)| idx); + parquet_column(parquet_schema, arrow_schema, name).map(|(idx, _field)| idx); assert_eq!(converter.parquet_column_index(), parquet_column_index); assert_eq!(converter.arrow_field().name(), name);