You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "alamb (via GitHub)" <gi...@apache.org> on 2023/11/21 18:04:58 UTC

[PR] Extract parquet statistics to its own module, add tests [arrow-datafusion]

alamb opened a new pull request, #8294:
URL: https://github.com/apache/arrow-datafusion/pull/8294

   ## Which issue does this PR close?
   
   Part of https://github.com/apache/arrow-datafusion/issues/8229
   
   
   In order to avoid boiling the ocean and to document more clearly what the current code does, this PR:
   
   
   ## Rationale for this change
   I am in the process of trying to improve the statistics in DataFusion, which have grown organically over time. I would like to refactor them, but I need to ensure that I don't break anything.
   
   There are tests for the existing pruning predicate code, but not the underlying statistics conversion.
   
   There are a few problems with the existing code:
   1. There are at least two copies of code that converts parquet statistics into DataFusion statistics, which have somewhat different semantics (TODO links)
   2. Currentky the statistics areconverted one row at a time (as an Array), which is not ideal from evaluating the pruning statistics
   
   <!--
    Why are you proposing this change? If this is already explained clearly in the issue then this section is not needed.
    Explaining clearly why changes are proposed helps reviewers understand your changes and offer better suggestions for fixes.  
   -->
   
   ## What changes are included in this PR?
   
   1. Extracts the statistics conversion code to a new module, and add a columnar API (returns value as an ArrayRef).
   2. Adds extensive tests, both round tripping data through `parquet` rust writer as well as using the existing parquet test data
   
   
   <!--
   There is no need to duplicate the description in the issue here but it is sometimes worth providing a summary of the individual changes in this PR.
   -->
   
   ## Are these changes tested?
   Yes
   <!--
   We typically require tests for all PRs in order to:
   1. Prevent the code from being accidentally broken by subsequent changes
   2. Serve as another way to document the expected behavior of the code
   
   If tests are not included in your PR, please explain why (for example, are they covered by existing tests)?
   -->
   
   ## Are there any user-facing changes?
   There are non intended. 
   
   This  implementation uses the same the existing code, so it is not a functional change, but it does add many tests for the existing code.
   
   I plan to improve the existing code in follow on PRs.
   
   <!--
   If there are user-facing changes then we may require documentation to be updated before approving the PR.
   -->
   
   <!--
   If there are any breaking changes to public APIs, please add the `api change` label.
   -->
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Extract parquet statistics to its own module, add tests [arrow-datafusion]

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on code in PR #8294:
URL: https://github.com/apache/arrow-datafusion/pull/8294#discussion_r1406589780


##########
datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs:
##########
@@ -431,11 +320,29 @@ macro_rules! get_null_count_values {
 
 impl<'a> PruningStatistics for RowGroupPruningStatistics<'a> {
     fn min_values(&self, column: &Column) -> Option<ArrayRef> {
-        get_min_max_values!(self, column, min, min_bytes)
+        let field = self
+            .parquet_schema
+            .fields()
+            .find(&column.name)
+            .map(|(_idx, field)| field)?;
+
+        RowGroupStatisticsConverter::new(field)

Review Comment:
   TLDR is that [Column](https://docs.rs/datafusion/latest/datafusion/common/struct.Column.html) does not address nested fields. The structure that does is 
   
   [datafusion_physical_expr](https://docs.rs/datafusion-physical-expr/33.0.0/datafusion_physical_expr/index.html)::[expressions](https://docs.rs/datafusion-physical-expr/33.0.0/datafusion_physical_expr/expressions/index.html)::[GetFieldAccessExpr](https://docs.rs/datafusion-physical-expr/33.0.0/datafusion_physical_expr/expressions/enum.GetFieldAccessExpr.html#)
   
   or 
   
   https://docs.rs/datafusion/latest/datafusion/logical_expr/expr/enum.Expr.html#variant.GetIndexedField



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Extract parquet statistics to its own module, add tests [arrow-datafusion]

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on code in PR #8294:
URL: https://github.com/apache/arrow-datafusion/pull/8294#discussion_r1406736395


##########
datafusion/core/src/datasource/physical_plan/parquet/statistics.rs:
##########
@@ -0,0 +1,807 @@
+// 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.
+
+//! [`RowGroupStatisticsConverter`] tp converts parquet RowGroup statistics to arrow [`ArrayRef`].
+
+use arrow::{array::ArrayRef, datatypes::DataType};
+use arrow_array::new_empty_array;
+use arrow_schema::Field;
+use datafusion_common::{Result, ScalarValue};
+use parquet::file::{
+    metadata::RowGroupMetaData, statistics::Statistics as ParquetStatistics,
+};
+
+// 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(b))
+}
+
+// Copy from arrow-rs
+// https://github.com/apache/arrow-rs/blob/733b7e7fd1e8c43a404c3ce40ecf741d493c21b4/parquet/src/arrow/buffer/bit_util.rs#L55
+// Convert the byte slice to fixed length byte array with the length of 16
+fn sign_extend_be(b: &[u8]) -> [u8; 16] {
+    assert!(b.len() <= 16, "Array too large, expected less than 16");
+    let is_negative = (b[0] & 128u8) == 128u8;
+    let mut result = if is_negative { [255u8; 16] } else { [0u8; 16] };
+    for (d, s) in result.iter_mut().skip(16 - b.len()).zip(b) {
+        *d = *s;
+    }
+    result
+}
+
+/// Converts parquet RowGroup statistics (stored in
+/// [`RowGroupMetaData`]) into an arrow [`ArrayRef`]
+///
+/// For example, given a parquet file with 3 Row Groups, when asked for
+/// statistics for column "A" it will return a single array with 3 elements,
+///
+pub(crate) struct RowGroupStatisticsConverter<'a> {
+    field: &'a Field,
+}
+
+/// Extract a single min/max statistics from a [`ParquetStatistics`] object
+///
+/// * `$column_statistics` is the `ParquetStatistics` object
+/// * `$func is the function` (`min`/`max`) to call to get the value
+/// * `$bytes_func` is the function (`min_bytes`/`max_bytes`) to call to get the value as bytes
+/// * `$target_arrow_type` is the [`DataType`] of the target statistics
+macro_rules! get_statistic {
+    ($column_statistics:expr, $func:ident, $bytes_func:ident, $target_arrow_type:expr) => {{
+        if !$column_statistics.has_min_max_set() {
+            return None;
+        }
+        match $column_statistics {
+            ParquetStatistics::Boolean(s) => Some(ScalarValue::Boolean(Some(*s.$func()))),
+            ParquetStatistics::Int32(s) => {
+                match $target_arrow_type {
+                    // int32 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int32(Some(*s.$func()))),
+                }
+            }
+            ParquetStatistics::Int64(s) => {
+                match $target_arrow_type {
+                    // int64 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int64(Some(*s.$func()))),
+                }
+            }
+            // 96 bit ints not supported
+            ParquetStatistics::Int96(_) => None,
+            ParquetStatistics::Float(s) => Some(ScalarValue::Float32(Some(*s.$func()))),
+            ParquetStatistics::Double(s) => Some(ScalarValue::Float64(Some(*s.$func()))),
+            ParquetStatistics::ByteArray(s) => {
+                match $target_arrow_type {
+                    // decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => {
+                        let s = std::str::from_utf8(s.$bytes_func())
+                            .map(|s| s.to_string())
+                            .ok();
+                        Some(ScalarValue::Utf8(s))
+                    }
+                }
+            }
+            // type not supported yet
+            ParquetStatistics::FixedLenByteArray(s) => {
+                match $target_arrow_type {
+                    // just support the decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => None,
+                }
+            }
+        }
+    }};
+}
+
+#[derive(Debug, Clone, Copy)]
+enum MinMax {

Review Comment:
   done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Extract parquet statistics to its own module, add tests [arrow-datafusion]

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on code in PR #8294:
URL: https://github.com/apache/arrow-datafusion/pull/8294#discussion_r1406745661


##########
datafusion/core/src/datasource/physical_plan/parquet/statistics.rs:
##########
@@ -0,0 +1,807 @@
+// 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.
+
+//! [`RowGroupStatisticsConverter`] tp converts parquet RowGroup statistics to arrow [`ArrayRef`].
+
+use arrow::{array::ArrayRef, datatypes::DataType};
+use arrow_array::new_empty_array;
+use arrow_schema::Field;
+use datafusion_common::{Result, ScalarValue};
+use parquet::file::{
+    metadata::RowGroupMetaData, statistics::Statistics as ParquetStatistics,
+};
+
+// 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(b))
+}
+
+// Copy from arrow-rs
+// https://github.com/apache/arrow-rs/blob/733b7e7fd1e8c43a404c3ce40ecf741d493c21b4/parquet/src/arrow/buffer/bit_util.rs#L55
+// Convert the byte slice to fixed length byte array with the length of 16
+fn sign_extend_be(b: &[u8]) -> [u8; 16] {
+    assert!(b.len() <= 16, "Array too large, expected less than 16");
+    let is_negative = (b[0] & 128u8) == 128u8;
+    let mut result = if is_negative { [255u8; 16] } else { [0u8; 16] };
+    for (d, s) in result.iter_mut().skip(16 - b.len()).zip(b) {
+        *d = *s;
+    }
+    result
+}
+
+/// Converts parquet RowGroup statistics (stored in
+/// [`RowGroupMetaData`]) into an arrow [`ArrayRef`]
+///
+/// For example, given a parquet file with 3 Row Groups, when asked for
+/// statistics for column "A" it will return a single array with 3 elements,
+///
+pub(crate) struct RowGroupStatisticsConverter<'a> {
+    field: &'a Field,
+}
+
+/// Extract a single min/max statistics from a [`ParquetStatistics`] object
+///
+/// * `$column_statistics` is the `ParquetStatistics` object
+/// * `$func is the function` (`min`/`max`) to call to get the value
+/// * `$bytes_func` is the function (`min_bytes`/`max_bytes`) to call to get the value as bytes
+/// * `$target_arrow_type` is the [`DataType`] of the target statistics
+macro_rules! get_statistic {
+    ($column_statistics:expr, $func:ident, $bytes_func:ident, $target_arrow_type:expr) => {{
+        if !$column_statistics.has_min_max_set() {
+            return None;
+        }
+        match $column_statistics {
+            ParquetStatistics::Boolean(s) => Some(ScalarValue::Boolean(Some(*s.$func()))),
+            ParquetStatistics::Int32(s) => {
+                match $target_arrow_type {
+                    // int32 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int32(Some(*s.$func()))),
+                }
+            }
+            ParquetStatistics::Int64(s) => {
+                match $target_arrow_type {
+                    // int64 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int64(Some(*s.$func()))),
+                }
+            }
+            // 96 bit ints not supported
+            ParquetStatistics::Int96(_) => None,
+            ParquetStatistics::Float(s) => Some(ScalarValue::Float32(Some(*s.$func()))),
+            ParquetStatistics::Double(s) => Some(ScalarValue::Float64(Some(*s.$func()))),
+            ParquetStatistics::ByteArray(s) => {
+                match $target_arrow_type {
+                    // decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => {
+                        let s = std::str::from_utf8(s.$bytes_func())
+                            .map(|s| s.to_string())
+                            .ok();
+                        Some(ScalarValue::Utf8(s))
+                    }
+                }
+            }
+            // type not supported yet
+            ParquetStatistics::FixedLenByteArray(s) => {
+                match $target_arrow_type {
+                    // just support the decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => None,
+                }
+            }
+        }
+    }};
+}
+
+#[derive(Debug, Clone, Copy)]
+enum MinMax {
+    Min,
+    Max,
+}
+
+impl<'a> RowGroupStatisticsConverter<'a> {
+    /// Create a new RowGoupStatisticsConverter suitable that can extract
+    /// statistics for the specified field
+    pub fn new(field: &'a Field) -> Self {
+        Self { field }
+    }
+
+    /// Returns the min value for the column into an array ref.
+    pub fn min<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(MinMax::Min, row_group_meta_data)
+    }
+
+    /// Returns the max value for the column into an array ref.
+    pub fn max<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(MinMax::Max, row_group_meta_data)
+    }
+
+    /// Extracts all min/max values for the column into an array ref.
+    fn min_max_impl<'b>(
+        &self,
+        mm: MinMax,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        let mut row_group_meta_data = row_group_meta_data.into_iter().peekable();
+
+        // if it is empty, return empty array
+        if row_group_meta_data.peek().is_none() {
+            return Ok(new_empty_array(self.field.data_type()));
+        }
+
+        let maybe_index = row_group_meta_data.peek().and_then(|rg_meta| {
+            rg_meta
+                .columns()
+                .iter()
+                .enumerate()
+                .find(|(_idx, c)| c.column_descr().name() == self.field.name())

Review Comment:
   Left a comment on how to see the bug



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Extract parquet statistics to its own module, add tests [arrow-datafusion]

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on code in PR #8294:
URL: https://github.com/apache/arrow-datafusion/pull/8294#discussion_r1406752147


##########
datafusion/core/src/datasource/physical_plan/parquet/statistics.rs:
##########
@@ -0,0 +1,807 @@
+// 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.
+
+//! [`RowGroupStatisticsConverter`] tp converts parquet RowGroup statistics to arrow [`ArrayRef`].
+
+use arrow::{array::ArrayRef, datatypes::DataType};
+use arrow_array::new_empty_array;
+use arrow_schema::Field;
+use datafusion_common::{Result, ScalarValue};
+use parquet::file::{
+    metadata::RowGroupMetaData, statistics::Statistics as ParquetStatistics,
+};
+
+// 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(b))
+}
+
+// Copy from arrow-rs
+// https://github.com/apache/arrow-rs/blob/733b7e7fd1e8c43a404c3ce40ecf741d493c21b4/parquet/src/arrow/buffer/bit_util.rs#L55
+// Convert the byte slice to fixed length byte array with the length of 16
+fn sign_extend_be(b: &[u8]) -> [u8; 16] {
+    assert!(b.len() <= 16, "Array too large, expected less than 16");
+    let is_negative = (b[0] & 128u8) == 128u8;
+    let mut result = if is_negative { [255u8; 16] } else { [0u8; 16] };
+    for (d, s) in result.iter_mut().skip(16 - b.len()).zip(b) {
+        *d = *s;
+    }
+    result
+}
+
+/// Converts parquet RowGroup statistics (stored in
+/// [`RowGroupMetaData`]) into an arrow [`ArrayRef`]
+///
+/// For example, given a parquet file with 3 Row Groups, when asked for
+/// statistics for column "A" it will return a single array with 3 elements,
+///
+pub(crate) struct RowGroupStatisticsConverter<'a> {
+    field: &'a Field,
+}
+
+/// Extract a single min/max statistics from a [`ParquetStatistics`] object
+///
+/// * `$column_statistics` is the `ParquetStatistics` object
+/// * `$func is the function` (`min`/`max`) to call to get the value
+/// * `$bytes_func` is the function (`min_bytes`/`max_bytes`) to call to get the value as bytes
+/// * `$target_arrow_type` is the [`DataType`] of the target statistics
+macro_rules! get_statistic {
+    ($column_statistics:expr, $func:ident, $bytes_func:ident, $target_arrow_type:expr) => {{
+        if !$column_statistics.has_min_max_set() {
+            return None;
+        }
+        match $column_statistics {
+            ParquetStatistics::Boolean(s) => Some(ScalarValue::Boolean(Some(*s.$func()))),
+            ParquetStatistics::Int32(s) => {
+                match $target_arrow_type {
+                    // int32 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int32(Some(*s.$func()))),
+                }
+            }
+            ParquetStatistics::Int64(s) => {
+                match $target_arrow_type {
+                    // int64 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int64(Some(*s.$func()))),
+                }
+            }
+            // 96 bit ints not supported
+            ParquetStatistics::Int96(_) => None,
+            ParquetStatistics::Float(s) => Some(ScalarValue::Float32(Some(*s.$func()))),
+            ParquetStatistics::Double(s) => Some(ScalarValue::Float64(Some(*s.$func()))),
+            ParquetStatistics::ByteArray(s) => {
+                match $target_arrow_type {
+                    // decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => {
+                        let s = std::str::from_utf8(s.$bytes_func())
+                            .map(|s| s.to_string())
+                            .ok();
+                        Some(ScalarValue::Utf8(s))
+                    }
+                }
+            }
+            // type not supported yet
+            ParquetStatistics::FixedLenByteArray(s) => {
+                match $target_arrow_type {
+                    // just support the decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => None,
+                }
+            }
+        }
+    }};
+}
+
+#[derive(Debug, Clone, Copy)]
+enum MinMax {
+    Min,
+    Max,
+}
+
+impl<'a> RowGroupStatisticsConverter<'a> {

Review Comment:
   > columns by their name (in the arrow Schema)
   
   The problem is this is not how parquet addresses the columns, and trying to pretend otherwise is going to result in a number of peculiar bugs. This mapping needs to exist somewhere, but the lowest level API should be in terms of leaf columns in order that it can meaningfully interpret the parquet logical types, especially if the eventual goal is to upstream this 
   
   That's not to say there can't be some higher level API, possibly PruningPredicate itself, that handles the parquet -> arrow mapping, but intermixing the parquet and arrow data models is just going to end up in a mess



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Extract parquet statistics to its own module, add tests [arrow-datafusion]

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on code in PR #8294:
URL: https://github.com/apache/arrow-datafusion/pull/8294#discussion_r1406752147


##########
datafusion/core/src/datasource/physical_plan/parquet/statistics.rs:
##########
@@ -0,0 +1,807 @@
+// 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.
+
+//! [`RowGroupStatisticsConverter`] tp converts parquet RowGroup statistics to arrow [`ArrayRef`].
+
+use arrow::{array::ArrayRef, datatypes::DataType};
+use arrow_array::new_empty_array;
+use arrow_schema::Field;
+use datafusion_common::{Result, ScalarValue};
+use parquet::file::{
+    metadata::RowGroupMetaData, statistics::Statistics as ParquetStatistics,
+};
+
+// 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(b))
+}
+
+// Copy from arrow-rs
+// https://github.com/apache/arrow-rs/blob/733b7e7fd1e8c43a404c3ce40ecf741d493c21b4/parquet/src/arrow/buffer/bit_util.rs#L55
+// Convert the byte slice to fixed length byte array with the length of 16
+fn sign_extend_be(b: &[u8]) -> [u8; 16] {
+    assert!(b.len() <= 16, "Array too large, expected less than 16");
+    let is_negative = (b[0] & 128u8) == 128u8;
+    let mut result = if is_negative { [255u8; 16] } else { [0u8; 16] };
+    for (d, s) in result.iter_mut().skip(16 - b.len()).zip(b) {
+        *d = *s;
+    }
+    result
+}
+
+/// Converts parquet RowGroup statistics (stored in
+/// [`RowGroupMetaData`]) into an arrow [`ArrayRef`]
+///
+/// For example, given a parquet file with 3 Row Groups, when asked for
+/// statistics for column "A" it will return a single array with 3 elements,
+///
+pub(crate) struct RowGroupStatisticsConverter<'a> {
+    field: &'a Field,
+}
+
+/// Extract a single min/max statistics from a [`ParquetStatistics`] object
+///
+/// * `$column_statistics` is the `ParquetStatistics` object
+/// * `$func is the function` (`min`/`max`) to call to get the value
+/// * `$bytes_func` is the function (`min_bytes`/`max_bytes`) to call to get the value as bytes
+/// * `$target_arrow_type` is the [`DataType`] of the target statistics
+macro_rules! get_statistic {
+    ($column_statistics:expr, $func:ident, $bytes_func:ident, $target_arrow_type:expr) => {{
+        if !$column_statistics.has_min_max_set() {
+            return None;
+        }
+        match $column_statistics {
+            ParquetStatistics::Boolean(s) => Some(ScalarValue::Boolean(Some(*s.$func()))),
+            ParquetStatistics::Int32(s) => {
+                match $target_arrow_type {
+                    // int32 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int32(Some(*s.$func()))),
+                }
+            }
+            ParquetStatistics::Int64(s) => {
+                match $target_arrow_type {
+                    // int64 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int64(Some(*s.$func()))),
+                }
+            }
+            // 96 bit ints not supported
+            ParquetStatistics::Int96(_) => None,
+            ParquetStatistics::Float(s) => Some(ScalarValue::Float32(Some(*s.$func()))),
+            ParquetStatistics::Double(s) => Some(ScalarValue::Float64(Some(*s.$func()))),
+            ParquetStatistics::ByteArray(s) => {
+                match $target_arrow_type {
+                    // decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => {
+                        let s = std::str::from_utf8(s.$bytes_func())
+                            .map(|s| s.to_string())
+                            .ok();
+                        Some(ScalarValue::Utf8(s))
+                    }
+                }
+            }
+            // type not supported yet
+            ParquetStatistics::FixedLenByteArray(s) => {
+                match $target_arrow_type {
+                    // just support the decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => None,
+                }
+            }
+        }
+    }};
+}
+
+#[derive(Debug, Clone, Copy)]
+enum MinMax {
+    Min,
+    Max,
+}
+
+impl<'a> RowGroupStatisticsConverter<'a> {

Review Comment:
   > columns by their name (in the arrow Schema)
   
   The problem is this is not how parquet addresses the columns, and trying to pretend otherwise is going to result in a number of peculiar bugs. This mapping needs to exist somewhere, but the lowest level API needs to be in terms of leaf columns in order that it can meaningfully interpret the parquet logical types. 
   
   That's not to say there can't be some higher level API, possibly PruningPredicate itself, that handles the parquet -> arrow mapping, but intermixing the parquet and arrow data models is just going to end up in a mess



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Extract parquet statistics to its own module, add tests [arrow-datafusion]

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on code in PR #8294:
URL: https://github.com/apache/arrow-datafusion/pull/8294#discussion_r1400976603


##########
datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs:
##########
@@ -303,112 +298,6 @@ struct RowGroupPruningStatistics<'a> {
     parquet_schema: &'a Schema,
 }
 
-/// Extract the min/max statistics from a `ParquetStatistics` object
-macro_rules! get_statistic {

Review Comment:
   This macro is moved, without modification, into `statistics.rs`



##########
datafusion/core/src/datasource/physical_plan/parquet.rs:
##########
@@ -718,28 +719,6 @@ pub async fn plan_to_parquet(
     Ok(())
 }
 
-// Copy from the arrow-rs

Review Comment:
   Moved to statistics.rs



##########
datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs:
##########
@@ -431,11 +320,29 @@ macro_rules! get_null_count_values {
 
 impl<'a> PruningStatistics for RowGroupPruningStatistics<'a> {
     fn min_values(&self, column: &Column) -> Option<ArrayRef> {
-        get_min_max_values!(self, column, min, min_bytes)
+        let field = self
+            .parquet_schema
+            .fields()
+            .find(&column.name)
+            .map(|(_idx, field)| field)?;
+
+        RowGoupStatisticsConverter::new(&field)

Review Comment:
   The idea here is (eventually) to prune more than one row group at a time. However, this PR still does it one at a time



##########
datafusion/core/src/datasource/physical_plan/parquet/statistics.rs:
##########
@@ -0,0 +1,805 @@
+// 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::ArrayRef, datatypes::DataType};
+use arrow_array::new_empty_array;
+use arrow_schema::Field;
+use datafusion_common::{Result, ScalarValue};
+use parquet::file::{
+    metadata::RowGroupMetaData, statistics::Statistics as ParquetStatistics,
+};
+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(b))
+}
+
+// Copy from arrow-rs
+// https://github.com/apache/arrow-rs/blob/733b7e7fd1e8c43a404c3ce40ecf741d493c21b4/parquet/src/arrow/buffer/bit_util.rs#L55
+// Convert the byte slice to fixed length byte array with the length of 16
+fn sign_extend_be(b: &[u8]) -> [u8; 16] {
+    assert!(b.len() <= 16, "Array too large, expected less than 16");
+    let is_negative = (b[0] & 128u8) == 128u8;
+    let mut result = if is_negative { [255u8; 16] } else { [0u8; 16] };
+    for (d, s) in result.iter_mut().skip(16 - b.len()).zip(b) {
+        *d = *s;
+    }
+    result
+}
+
+/// Converts parquet RowGroup statistics (stored in
+/// [`RowGroupMetaData`]) into an arrow [`ArrayRef`]
+///
+/// For example, given a parquet file with 3 Row Groups, when asked for
+/// statistics for column "A" it will return a single array with 3 elements,
+///
+pub(crate) struct RowGoupStatisticsConverter<'a> {
+    field: &'a Field,
+}
+
+/// Extract a single min/max statistics from a [`ParquetStatistics`] object
+///
+/// * `$column_statistics` is the `ParquetStatistics` object
+/// * `$func is the function` (`min`/`max`) to call to get the value
+/// * `$bytes_func` is the function (`min_bytes`/`max_bytes`) to call to get the value as bytes
+/// * `$target_arrow_type` is the [`DataType`] of the target statistics
+macro_rules! get_statistic {
+    ($column_statistics:expr, $func:ident, $bytes_func:ident, $target_arrow_type:expr) => {{
+        if !$column_statistics.has_min_max_set() {
+            return None;
+        }
+        match $column_statistics {
+            ParquetStatistics::Boolean(s) => Some(ScalarValue::Boolean(Some(*s.$func()))),
+            ParquetStatistics::Int32(s) => {
+                match $target_arrow_type {
+                    // int32 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int32(Some(*s.$func()))),
+                }
+            }
+            ParquetStatistics::Int64(s) => {
+                match $target_arrow_type {
+                    // int64 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int64(Some(*s.$func()))),
+                }
+            }
+            // 96 bit ints not supported
+            ParquetStatistics::Int96(_) => None,
+            ParquetStatistics::Float(s) => Some(ScalarValue::Float32(Some(*s.$func()))),
+            ParquetStatistics::Double(s) => Some(ScalarValue::Float64(Some(*s.$func()))),
+            ParquetStatistics::ByteArray(s) => {
+                match $target_arrow_type {
+                    // decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => {
+                        let s = std::str::from_utf8(s.$bytes_func())
+                            .map(|s| s.to_string())
+                            .ok();
+                        Some(ScalarValue::Utf8(s))
+                    }
+                }
+            }
+            // type not supported yet
+            ParquetStatistics::FixedLenByteArray(s) => {
+                match $target_arrow_type {
+                    // just support the decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => None,
+                }
+            }
+        }
+    }};
+}
+
+#[derive(Debug, Clone, Copy)]
+enum MinMax {
+    Min,
+    Max,
+}
+
+impl<'a> RowGoupStatisticsConverter<'a> {
+    /// Create a new RowGoupStatisticsConverter suitable that can extract
+    /// statistics for the specified field
+    pub fn new(field: &'a Field) -> Self {
+        Self { field }
+    }
+
+    /// Returns the min value for the column into an array ref.
+    pub fn min<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(MinMax::Min, row_group_meta_data)
+    }
+
+    /// Returns the max value for the column into an array ref.
+    pub fn max<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(MinMax::Max, row_group_meta_data)
+    }
+
+    /// Extracts all min/max values for the column into an array ref.
+    fn min_max_impl<'b>(
+        &self,
+        mm: MinMax,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        let mut row_group_meta_data = row_group_meta_data.into_iter().peekable();
+
+        // if it is empty, return empty array

Review Comment:
   this empty handling is new, to support the new array ref interface



##########
datafusion/core/src/datasource/physical_plan/parquet/statistics.rs:
##########
@@ -0,0 +1,805 @@
+// 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::ArrayRef, datatypes::DataType};
+use arrow_array::new_empty_array;
+use arrow_schema::Field;
+use datafusion_common::{Result, ScalarValue};
+use parquet::file::{
+    metadata::RowGroupMetaData, statistics::Statistics as ParquetStatistics,
+};
+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(b))
+}
+
+// Copy from arrow-rs
+// https://github.com/apache/arrow-rs/blob/733b7e7fd1e8c43a404c3ce40ecf741d493c21b4/parquet/src/arrow/buffer/bit_util.rs#L55
+// Convert the byte slice to fixed length byte array with the length of 16
+fn sign_extend_be(b: &[u8]) -> [u8; 16] {
+    assert!(b.len() <= 16, "Array too large, expected less than 16");
+    let is_negative = (b[0] & 128u8) == 128u8;
+    let mut result = if is_negative { [255u8; 16] } else { [0u8; 16] };
+    for (d, s) in result.iter_mut().skip(16 - b.len()).zip(b) {
+        *d = *s;
+    }
+    result
+}
+
+/// Converts parquet RowGroup statistics (stored in
+/// [`RowGroupMetaData`]) into an arrow [`ArrayRef`]
+///
+/// For example, given a parquet file with 3 Row Groups, when asked for
+/// statistics for column "A" it will return a single array with 3 elements,
+///
+pub(crate) struct RowGoupStatisticsConverter<'a> {
+    field: &'a Field,
+}
+
+/// Extract a single min/max statistics from a [`ParquetStatistics`] object
+///
+/// * `$column_statistics` is the `ParquetStatistics` object
+/// * `$func is the function` (`min`/`max`) to call to get the value
+/// * `$bytes_func` is the function (`min_bytes`/`max_bytes`) to call to get the value as bytes
+/// * `$target_arrow_type` is the [`DataType`] of the target statistics
+macro_rules! get_statistic {

Review Comment:
   This implementation leaves a lot to be desired, but I want to get tests in place before I start changing it



##########
datafusion/core/src/datasource/physical_plan/parquet/statistics.rs:
##########
@@ -0,0 +1,805 @@
+// 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::ArrayRef, datatypes::DataType};
+use arrow_array::new_empty_array;
+use arrow_schema::Field;
+use datafusion_common::{Result, ScalarValue};
+use parquet::file::{
+    metadata::RowGroupMetaData, statistics::Statistics as ParquetStatistics,
+};
+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(b))
+}
+
+// Copy from arrow-rs
+// https://github.com/apache/arrow-rs/blob/733b7e7fd1e8c43a404c3ce40ecf741d493c21b4/parquet/src/arrow/buffer/bit_util.rs#L55
+// Convert the byte slice to fixed length byte array with the length of 16
+fn sign_extend_be(b: &[u8]) -> [u8; 16] {
+    assert!(b.len() <= 16, "Array too large, expected less than 16");
+    let is_negative = (b[0] & 128u8) == 128u8;
+    let mut result = if is_negative { [255u8; 16] } else { [0u8; 16] };
+    for (d, s) in result.iter_mut().skip(16 - b.len()).zip(b) {
+        *d = *s;
+    }
+    result
+}
+
+/// Converts parquet RowGroup statistics (stored in
+/// [`RowGroupMetaData`]) into an arrow [`ArrayRef`]
+///
+/// For example, given a parquet file with 3 Row Groups, when asked for
+/// statistics for column "A" it will return a single array with 3 elements,
+///
+pub(crate) struct RowGoupStatisticsConverter<'a> {
+    field: &'a Field,
+}
+
+/// Extract a single min/max statistics from a [`ParquetStatistics`] object
+///
+/// * `$column_statistics` is the `ParquetStatistics` object
+/// * `$func is the function` (`min`/`max`) to call to get the value
+/// * `$bytes_func` is the function (`min_bytes`/`max_bytes`) to call to get the value as bytes
+/// * `$target_arrow_type` is the [`DataType`] of the target statistics
+macro_rules! get_statistic {
+    ($column_statistics:expr, $func:ident, $bytes_func:ident, $target_arrow_type:expr) => {{
+        if !$column_statistics.has_min_max_set() {
+            return None;
+        }
+        match $column_statistics {
+            ParquetStatistics::Boolean(s) => Some(ScalarValue::Boolean(Some(*s.$func()))),
+            ParquetStatistics::Int32(s) => {
+                match $target_arrow_type {
+                    // int32 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int32(Some(*s.$func()))),
+                }
+            }
+            ParquetStatistics::Int64(s) => {
+                match $target_arrow_type {
+                    // int64 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int64(Some(*s.$func()))),
+                }
+            }
+            // 96 bit ints not supported
+            ParquetStatistics::Int96(_) => None,
+            ParquetStatistics::Float(s) => Some(ScalarValue::Float32(Some(*s.$func()))),
+            ParquetStatistics::Double(s) => Some(ScalarValue::Float64(Some(*s.$func()))),
+            ParquetStatistics::ByteArray(s) => {
+                match $target_arrow_type {
+                    // decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => {
+                        let s = std::str::from_utf8(s.$bytes_func())
+                            .map(|s| s.to_string())
+                            .ok();
+                        Some(ScalarValue::Utf8(s))
+                    }
+                }
+            }
+            // type not supported yet
+            ParquetStatistics::FixedLenByteArray(s) => {
+                match $target_arrow_type {
+                    // just support the decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => None,
+                }
+            }
+        }
+    }};
+}
+
+#[derive(Debug, Clone, Copy)]
+enum MinMax {
+    Min,
+    Max,
+}
+
+impl<'a> RowGoupStatisticsConverter<'a> {
+    /// Create a new RowGoupStatisticsConverter suitable that can extract
+    /// statistics for the specified field
+    pub fn new(field: &'a Field) -> Self {
+        Self { field }
+    }
+
+    /// Returns the min value for the column into an array ref.
+    pub fn min<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(MinMax::Min, row_group_meta_data)
+    }
+
+    /// Returns the max value for the column into an array ref.
+    pub fn max<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(MinMax::Max, row_group_meta_data)
+    }
+
+    /// Extracts all min/max values for the column into an array ref.
+    fn min_max_impl<'b>(
+        &self,
+        mm: MinMax,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        let mut row_group_meta_data = row_group_meta_data.into_iter().peekable();
+
+        // if it is empty, return empty array
+        if row_group_meta_data.peek().is_none() {
+            return Ok(new_empty_array(self.field.data_type()));
+        }
+
+        let maybe_index = row_group_meta_data.peek().and_then(|rg_meta| {
+            rg_meta
+                .columns()
+                .iter()
+                .enumerate()
+                .find(|(_idx, c)| c.column_descr().name() == self.field.name())
+                .map(|(idx, _c)| idx)
+        });
+
+        // don't have this column, return an array of all NULLs
+        let Some(column_index) = maybe_index else {
+            let num_row_groups = row_group_meta_data.count();
+            let sv = ScalarValue::try_from(self.field.data_type())?;
+            return sv.to_array_of_size(num_row_groups);
+        };
+
+        let stats_iter = row_group_meta_data.map(move |row_group_meta_data| {
+            row_group_meta_data.column(column_index).statistics()
+        });
+
+        // this is the value to use when the statistics are not set
+        let null_value = ScalarValue::try_from(self.field.data_type())?;
+        match mm {
+            MinMax::Min => {
+                let values = stats_iter.map(|column_statistics| {
+                    column_statistics
+                        .and_then(|column_statistics| {
+                            get_statistic!(
+                                column_statistics,
+                                min,
+                                min_bytes,
+                                Some(self.field.data_type().clone())
+                            )
+                        })
+                        .unwrap_or_else(|| null_value.clone())
+                });
+                ScalarValue::iter_to_array(values)
+            }
+            MinMax::Max => {
+                let values = stats_iter.map(|column_statistics| {
+                    column_statistics
+                        .and_then(|column_statistics| {
+                            get_statistic!(
+                                column_statistics,
+                                max,
+                                max_bytes,
+                                Some(self.field.data_type().clone())
+                            )
+                        })
+                        .unwrap_or_else(|| null_value.clone())
+                });
+                ScalarValue::iter_to_array(values)
+            }
+        }
+    }
+}
+
+#[cfg(test)]
+mod test {
+    use super::*;
+    use arrow_array::{
+        BinaryArray, BooleanArray, Decimal128Array, Float32Array, Float64Array,
+        Int32Array, Int64Array, RecordBatch, StringArray, TimestampNanosecondArray,
+    };
+    use arrow_schema::SchemaRef;
+    use bytes::Bytes;
+    use datafusion_common::test_util::parquet_test_data;
+    use parquet::arrow::arrow_reader::ArrowReaderBuilder;
+    use parquet::arrow::arrow_writer::ArrowWriter;
+    use parquet::file::metadata::ParquetMetaData;
+    use parquet::file::properties::{EnabledStatistics, WriterProperties};
+    use std::path::PathBuf;
+
+    // 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: Arc::new(BooleanArray::from(vec![
+                // row group 1
+                Some(true),
+                None,
+                Some(true),
+                // row group 2
+                Some(true),
+                Some(false),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ])),
+            expected_min: Arc::new(BooleanArray::from(vec![
+                Some(true),
+                Some(false),
+                None,
+            ])),
+            expected_max: Arc::new(BooleanArray::from(vec![
+                Some(true),
+                Some(true),
+                None,
+            ])),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_int32() {
+        Test {
+            input: Arc::new(Int32Array::from(vec![
+                // row group 1
+                Some(1),
+                None,
+                Some(3),
+                // row group 2
+                Some(0),
+                Some(5),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ])),
+            expected_min: Arc::new(Int32Array::from(vec![Some(1), Some(0), None])),
+            expected_max: Arc::new(Int32Array::from(vec![Some(3), Some(5), None])),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_int64() {
+        Test {
+            input: Arc::new(Int64Array::from(vec![
+                // row group 1
+                Some(1),
+                None,
+                Some(3),
+                // row group 2
+                Some(0),
+                Some(5),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ])),
+            expected_min: Arc::new(Int64Array::from(vec![Some(1), Some(0), None])),
+            expected_max: Arc::new(Int64Array::from(vec![Some(3), Some(5), None])),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_f32() {
+        Test {
+            input: Arc::new(Float32Array::from(vec![
+                // 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: Arc::new(Float32Array::from(vec![Some(1.0), Some(-1.0), None])),
+            expected_max: Arc::new(Float32Array::from(vec![Some(3.0), Some(5.0), None])),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_f64() {
+        Test {
+            input: Arc::new(Float64Array::from(vec![
+                // 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: Arc::new(Float64Array::from(vec![Some(1.0), Some(-1.0), None])),
+            expected_max: Arc::new(Float64Array::from(vec![Some(3.0), Some(5.0), None])),
+        }
+        .run()
+    }
+
+    #[test]
+    #[should_panic(

Review Comment:
   It is not good that the statistics don't round trip -- I will write up a ticket prior to merging this PR



##########
datafusion/core/src/datasource/physical_plan/parquet/statistics.rs:
##########
@@ -0,0 +1,805 @@
+// 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::ArrayRef, datatypes::DataType};
+use arrow_array::new_empty_array;
+use arrow_schema::Field;
+use datafusion_common::{Result, ScalarValue};
+use parquet::file::{
+    metadata::RowGroupMetaData, statistics::Statistics as ParquetStatistics,
+};
+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(b))
+}
+
+// Copy from arrow-rs
+// https://github.com/apache/arrow-rs/blob/733b7e7fd1e8c43a404c3ce40ecf741d493c21b4/parquet/src/arrow/buffer/bit_util.rs#L55
+// Convert the byte slice to fixed length byte array with the length of 16
+fn sign_extend_be(b: &[u8]) -> [u8; 16] {
+    assert!(b.len() <= 16, "Array too large, expected less than 16");
+    let is_negative = (b[0] & 128u8) == 128u8;
+    let mut result = if is_negative { [255u8; 16] } else { [0u8; 16] };
+    for (d, s) in result.iter_mut().skip(16 - b.len()).zip(b) {
+        *d = *s;
+    }
+    result
+}
+
+/// Converts parquet RowGroup statistics (stored in
+/// [`RowGroupMetaData`]) into an arrow [`ArrayRef`]
+///
+/// For example, given a parquet file with 3 Row Groups, when asked for
+/// statistics for column "A" it will return a single array with 3 elements,
+///
+pub(crate) struct RowGoupStatisticsConverter<'a> {
+    field: &'a Field,
+}
+
+/// Extract a single min/max statistics from a [`ParquetStatistics`] object
+///
+/// * `$column_statistics` is the `ParquetStatistics` object
+/// * `$func is the function` (`min`/`max`) to call to get the value
+/// * `$bytes_func` is the function (`min_bytes`/`max_bytes`) to call to get the value as bytes
+/// * `$target_arrow_type` is the [`DataType`] of the target statistics
+macro_rules! get_statistic {
+    ($column_statistics:expr, $func:ident, $bytes_func:ident, $target_arrow_type:expr) => {{
+        if !$column_statistics.has_min_max_set() {
+            return None;
+        }
+        match $column_statistics {
+            ParquetStatistics::Boolean(s) => Some(ScalarValue::Boolean(Some(*s.$func()))),
+            ParquetStatistics::Int32(s) => {
+                match $target_arrow_type {
+                    // int32 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int32(Some(*s.$func()))),
+                }
+            }
+            ParquetStatistics::Int64(s) => {
+                match $target_arrow_type {
+                    // int64 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int64(Some(*s.$func()))),
+                }
+            }
+            // 96 bit ints not supported
+            ParquetStatistics::Int96(_) => None,
+            ParquetStatistics::Float(s) => Some(ScalarValue::Float32(Some(*s.$func()))),
+            ParquetStatistics::Double(s) => Some(ScalarValue::Float64(Some(*s.$func()))),
+            ParquetStatistics::ByteArray(s) => {
+                match $target_arrow_type {
+                    // decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => {
+                        let s = std::str::from_utf8(s.$bytes_func())
+                            .map(|s| s.to_string())
+                            .ok();
+                        Some(ScalarValue::Utf8(s))
+                    }
+                }
+            }
+            // type not supported yet
+            ParquetStatistics::FixedLenByteArray(s) => {
+                match $target_arrow_type {
+                    // just support the decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => None,
+                }
+            }
+        }
+    }};
+}
+
+#[derive(Debug, Clone, Copy)]
+enum MinMax {
+    Min,
+    Max,
+}
+
+impl<'a> RowGoupStatisticsConverter<'a> {
+    /// Create a new RowGoupStatisticsConverter suitable that can extract
+    /// statistics for the specified field
+    pub fn new(field: &'a Field) -> Self {
+        Self { field }
+    }
+
+    /// Returns the min value for the column into an array ref.
+    pub fn min<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(MinMax::Min, row_group_meta_data)
+    }
+
+    /// Returns the max value for the column into an array ref.
+    pub fn max<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(MinMax::Max, row_group_meta_data)
+    }
+
+    /// Extracts all min/max values for the column into an array ref.
+    fn min_max_impl<'b>(
+        &self,
+        mm: MinMax,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        let mut row_group_meta_data = row_group_meta_data.into_iter().peekable();
+
+        // if it is empty, return empty array
+        if row_group_meta_data.peek().is_none() {
+            return Ok(new_empty_array(self.field.data_type()));
+        }
+
+        let maybe_index = row_group_meta_data.peek().and_then(|rg_meta| {
+            rg_meta
+                .columns()
+                .iter()
+                .enumerate()
+                .find(|(_idx, c)| c.column_descr().name() == self.field.name())
+                .map(|(idx, _c)| idx)
+        });
+
+        // don't have this column, return an array of all NULLs
+        let Some(column_index) = maybe_index else {
+            let num_row_groups = row_group_meta_data.count();
+            let sv = ScalarValue::try_from(self.field.data_type())?;
+            return sv.to_array_of_size(num_row_groups);
+        };
+
+        let stats_iter = row_group_meta_data.map(move |row_group_meta_data| {
+            row_group_meta_data.column(column_index).statistics()
+        });
+
+        // this is the value to use when the statistics are not set
+        let null_value = ScalarValue::try_from(self.field.data_type())?;
+        match mm {
+            MinMax::Min => {
+                let values = stats_iter.map(|column_statistics| {
+                    column_statistics
+                        .and_then(|column_statistics| {
+                            get_statistic!(
+                                column_statistics,
+                                min,
+                                min_bytes,
+                                Some(self.field.data_type().clone())
+                            )
+                        })
+                        .unwrap_or_else(|| null_value.clone())
+                });
+                ScalarValue::iter_to_array(values)
+            }
+            MinMax::Max => {
+                let values = stats_iter.map(|column_statistics| {
+                    column_statistics
+                        .and_then(|column_statistics| {
+                            get_statistic!(
+                                column_statistics,
+                                max,
+                                max_bytes,
+                                Some(self.field.data_type().clone())
+                            )
+                        })
+                        .unwrap_or_else(|| null_value.clone())
+                });
+                ScalarValue::iter_to_array(values)
+            }
+        }
+    }
+}
+
+#[cfg(test)]
+mod test {
+    use super::*;
+    use arrow_array::{
+        BinaryArray, BooleanArray, Decimal128Array, Float32Array, Float64Array,
+        Int32Array, Int64Array, RecordBatch, StringArray, TimestampNanosecondArray,
+    };
+    use arrow_schema::SchemaRef;
+    use bytes::Bytes;
+    use datafusion_common::test_util::parquet_test_data;
+    use parquet::arrow::arrow_reader::ArrowReaderBuilder;
+    use parquet::arrow::arrow_writer::ArrowWriter;
+    use parquet::file::metadata::ParquetMetaData;
+    use parquet::file::properties::{EnabledStatistics, WriterProperties};
+    use std::path::PathBuf;
+
+    // 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: Arc::new(BooleanArray::from(vec![
+                // row group 1
+                Some(true),
+                None,
+                Some(true),
+                // row group 2
+                Some(true),
+                Some(false),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ])),
+            expected_min: Arc::new(BooleanArray::from(vec![
+                Some(true),
+                Some(false),
+                None,
+            ])),
+            expected_max: Arc::new(BooleanArray::from(vec![
+                Some(true),
+                Some(true),
+                None,
+            ])),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_int32() {
+        Test {
+            input: Arc::new(Int32Array::from(vec![
+                // row group 1
+                Some(1),
+                None,
+                Some(3),
+                // row group 2
+                Some(0),
+                Some(5),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ])),
+            expected_min: Arc::new(Int32Array::from(vec![Some(1), Some(0), None])),
+            expected_max: Arc::new(Int32Array::from(vec![Some(3), Some(5), None])),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_int64() {
+        Test {
+            input: Arc::new(Int64Array::from(vec![
+                // row group 1
+                Some(1),
+                None,
+                Some(3),
+                // row group 2
+                Some(0),
+                Some(5),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ])),
+            expected_min: Arc::new(Int64Array::from(vec![Some(1), Some(0), None])),
+            expected_max: Arc::new(Int64Array::from(vec![Some(3), Some(5), None])),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_f32() {
+        Test {
+            input: Arc::new(Float32Array::from(vec![
+                // 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: Arc::new(Float32Array::from(vec![Some(1.0), Some(-1.0), None])),
+            expected_max: Arc::new(Float32Array::from(vec![Some(3.0), Some(5.0), None])),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_f64() {
+        Test {
+            input: Arc::new(Float64Array::from(vec![
+                // 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: Arc::new(Float64Array::from(vec![Some(1.0), Some(-1.0), None])),
+            expected_max: Arc::new(Float64Array::from(vec![Some(3.0), Some(5.0), None])),
+        }
+        .run()
+    }
+
+    #[test]
+    #[should_panic(
+        expected = "Inconsistent types in ScalarValue::iter_to_array. Expected Int64, got TimestampNanosecond(NULL, None)"
+    )]
+    fn roundtrip_timestamp() {
+        Test {
+            input: Arc::new(TimestampNanosecondArray::from(vec![
+                // row group 1
+                Some(1),
+                None,
+                Some(3),
+                // row group 2
+                Some(9),
+                Some(5),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ])),
+            expected_min: Arc::new(TimestampNanosecondArray::from(vec![
+                Some(1),
+                Some(5),
+                None,
+            ])),
+            expected_max: Arc::new(TimestampNanosecondArray::from(vec![
+                Some(3),
+                Some(9),
+                None,
+            ])),
+        }
+        .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]
+    fn roundtrip_utf8() {
+        Test {
+            input: Arc::new(StringArray::from(vec![
+                // row group 1
+                Some("A"),
+                None,
+                Some("Q"),
+                // row group 2
+                Some("ZZ"),
+                Some("AA"),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ])),
+            expected_min: Arc::new(StringArray::from(vec![Some("A"), Some("AA"), None])),
+            expected_max: Arc::new(StringArray::from(vec![Some("Q"), Some("ZZ"), None])),
+        }
+        .run()
+    }
+
+    #[test]
+    #[should_panic(
+        expected = "Inconsistent types in ScalarValue::iter_to_array. Expected Utf8, got Binary(NULL)"

Review Comment:
   Likewise, this should round trip but does not



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Extract parquet statistics to its own module, add tests [arrow-datafusion]

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on code in PR #8294:
URL: https://github.com/apache/arrow-datafusion/pull/8294#discussion_r1402772669


##########
datafusion/core/src/datasource/physical_plan/parquet/statistics.rs:
##########
@@ -0,0 +1,807 @@
+// 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.
+
+//! [`RowGroupStatisticsConverter`] tp converts parquet RowGroup statistics to arrow [`ArrayRef`].
+
+use arrow::{array::ArrayRef, datatypes::DataType};
+use arrow_array::new_empty_array;
+use arrow_schema::Field;
+use datafusion_common::{Result, ScalarValue};
+use parquet::file::{
+    metadata::RowGroupMetaData, statistics::Statistics as ParquetStatistics,
+};
+
+// 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(b))
+}
+
+// Copy from arrow-rs
+// https://github.com/apache/arrow-rs/blob/733b7e7fd1e8c43a404c3ce40ecf741d493c21b4/parquet/src/arrow/buffer/bit_util.rs#L55
+// Convert the byte slice to fixed length byte array with the length of 16
+fn sign_extend_be(b: &[u8]) -> [u8; 16] {
+    assert!(b.len() <= 16, "Array too large, expected less than 16");
+    let is_negative = (b[0] & 128u8) == 128u8;
+    let mut result = if is_negative { [255u8; 16] } else { [0u8; 16] };
+    for (d, s) in result.iter_mut().skip(16 - b.len()).zip(b) {
+        *d = *s;
+    }
+    result
+}
+
+/// Converts parquet RowGroup statistics (stored in
+/// [`RowGroupMetaData`]) into an arrow [`ArrayRef`]
+///
+/// For example, given a parquet file with 3 Row Groups, when asked for
+/// statistics for column "A" it will return a single array with 3 elements,
+///
+pub(crate) struct RowGroupStatisticsConverter<'a> {
+    field: &'a Field,
+}
+
+/// Extract a single min/max statistics from a [`ParquetStatistics`] object
+///
+/// * `$column_statistics` is the `ParquetStatistics` object
+/// * `$func is the function` (`min`/`max`) to call to get the value
+/// * `$bytes_func` is the function (`min_bytes`/`max_bytes`) to call to get the value as bytes
+/// * `$target_arrow_type` is the [`DataType`] of the target statistics
+macro_rules! get_statistic {
+    ($column_statistics:expr, $func:ident, $bytes_func:ident, $target_arrow_type:expr) => {{
+        if !$column_statistics.has_min_max_set() {
+            return None;
+        }
+        match $column_statistics {
+            ParquetStatistics::Boolean(s) => Some(ScalarValue::Boolean(Some(*s.$func()))),
+            ParquetStatistics::Int32(s) => {
+                match $target_arrow_type {
+                    // int32 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int32(Some(*s.$func()))),
+                }
+            }
+            ParquetStatistics::Int64(s) => {
+                match $target_arrow_type {
+                    // int64 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int64(Some(*s.$func()))),
+                }
+            }
+            // 96 bit ints not supported
+            ParquetStatistics::Int96(_) => None,
+            ParquetStatistics::Float(s) => Some(ScalarValue::Float32(Some(*s.$func()))),
+            ParquetStatistics::Double(s) => Some(ScalarValue::Float64(Some(*s.$func()))),
+            ParquetStatistics::ByteArray(s) => {
+                match $target_arrow_type {
+                    // decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => {
+                        let s = std::str::from_utf8(s.$bytes_func())
+                            .map(|s| s.to_string())
+                            .ok();
+                        Some(ScalarValue::Utf8(s))
+                    }
+                }
+            }
+            // type not supported yet
+            ParquetStatistics::FixedLenByteArray(s) => {
+                match $target_arrow_type {
+                    // just support the decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => None,
+                }
+            }
+        }
+    }};
+}
+
+#[derive(Debug, Clone, Copy)]
+enum MinMax {
+    Min,
+    Max,
+}
+
+impl<'a> RowGroupStatisticsConverter<'a> {
+    /// Create a new RowGoupStatisticsConverter suitable that can extract
+    /// statistics for the specified field
+    pub fn new(field: &'a Field) -> Self {
+        Self { field }
+    }
+
+    /// Returns the min value for the column into an array ref.
+    pub fn min<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(MinMax::Min, row_group_meta_data)
+    }
+
+    /// Returns the max value for the column into an array ref.
+    pub fn max<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(MinMax::Max, row_group_meta_data)
+    }
+
+    /// Extracts all min/max values for the column into an array ref.
+    fn min_max_impl<'b>(
+        &self,
+        mm: MinMax,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        let mut row_group_meta_data = row_group_meta_data.into_iter().peekable();
+
+        // if it is empty, return empty array
+        if row_group_meta_data.peek().is_none() {
+            return Ok(new_empty_array(self.field.data_type()));
+        }
+
+        let maybe_index = row_group_meta_data.peek().and_then(|rg_meta| {
+            rg_meta
+                .columns()
+                .iter()
+                .enumerate()
+                .find(|(_idx, c)| c.column_descr().name() == self.field.name())
+                .map(|(idx, _c)| idx)
+        });
+
+        // don't have this column, return an array of all NULLs
+        let Some(column_index) = maybe_index else {
+            let num_row_groups = row_group_meta_data.count();
+            let sv = ScalarValue::try_from(self.field.data_type())?;
+            return sv.to_array_of_size(num_row_groups);
+        };
+
+        let stats_iter = row_group_meta_data.map(move |row_group_meta_data| {
+            row_group_meta_data.column(column_index).statistics()
+        });
+
+        // this is the value to use when the statistics are not set
+        let null_value = ScalarValue::try_from(self.field.data_type())?;
+        match mm {
+            MinMax::Min => {
+                let values = stats_iter.map(|column_statistics| {
+                    column_statistics
+                        .and_then(|column_statistics| {
+                            get_statistic!(
+                                column_statistics,
+                                min,
+                                min_bytes,
+                                Some(self.field.data_type().clone())
+                            )
+                        })
+                        .unwrap_or_else(|| null_value.clone())
+                });
+                ScalarValue::iter_to_array(values)
+            }
+            MinMax::Max => {
+                let values = stats_iter.map(|column_statistics| {
+                    column_statistics
+                        .and_then(|column_statistics| {
+                            get_statistic!(
+                                column_statistics,
+                                max,
+                                max_bytes,
+                                Some(self.field.data_type().clone())
+                            )
+                        })
+                        .unwrap_or_else(|| null_value.clone())
+                });
+                ScalarValue::iter_to_array(values)
+            }
+        }
+    }
+}
+
+#[cfg(test)]
+mod test {
+    use super::*;
+    use arrow_array::{
+        BinaryArray, BooleanArray, Decimal128Array, Float32Array, Float64Array,
+        Int32Array, Int64Array, RecordBatch, StringArray, TimestampNanosecondArray,
+    };
+    use arrow_schema::SchemaRef;
+    use bytes::Bytes;
+    use datafusion_common::test_util::parquet_test_data;
+    use parquet::arrow::arrow_reader::ArrowReaderBuilder;
+    use parquet::arrow::arrow_writer::ArrowWriter;
+    use parquet::file::metadata::ParquetMetaData;
+    use parquet::file::properties::{EnabledStatistics, WriterProperties};
+    use std::path::PathBuf;
+    use std::sync::Arc;
+
+    // TODO error cases (with parquet statistics that are mismatched in expected type)
+

Review Comment:
   Whilst I'm not entirely sure how repeated fields could be handled, I think a test of nested groups, i.e. StructArray, would be worthwhile to demonstrate the mismatch between arrow and parquet schema representations, and I think is a useful test.
   
   We could also have a test that repeated fields are correctly ignored or something



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Extract parquet statistics to its own module, add tests [arrow-datafusion]

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on PR #8294:
URL: https://github.com/apache/arrow-datafusion/pull/8294#issuecomment-1854761469

   This PR introduced a regression it turns out: https://github.com/apache/arrow-datafusion/pull/8533


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Extract parquet statistics to its own module, add tests [arrow-datafusion]

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on PR #8294:
URL: https://github.com/apache/arrow-datafusion/pull/8294#issuecomment-1823232086

   I would also be interested in opinions about potentially moving this implementation upstream into the parquet-rs eventally 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Extract parquet statistics to its own module, add tests [arrow-datafusion]

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on code in PR #8294:
URL: https://github.com/apache/arrow-datafusion/pull/8294#discussion_r1406752147


##########
datafusion/core/src/datasource/physical_plan/parquet/statistics.rs:
##########
@@ -0,0 +1,807 @@
+// 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.
+
+//! [`RowGroupStatisticsConverter`] tp converts parquet RowGroup statistics to arrow [`ArrayRef`].
+
+use arrow::{array::ArrayRef, datatypes::DataType};
+use arrow_array::new_empty_array;
+use arrow_schema::Field;
+use datafusion_common::{Result, ScalarValue};
+use parquet::file::{
+    metadata::RowGroupMetaData, statistics::Statistics as ParquetStatistics,
+};
+
+// 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(b))
+}
+
+// Copy from arrow-rs
+// https://github.com/apache/arrow-rs/blob/733b7e7fd1e8c43a404c3ce40ecf741d493c21b4/parquet/src/arrow/buffer/bit_util.rs#L55
+// Convert the byte slice to fixed length byte array with the length of 16
+fn sign_extend_be(b: &[u8]) -> [u8; 16] {
+    assert!(b.len() <= 16, "Array too large, expected less than 16");
+    let is_negative = (b[0] & 128u8) == 128u8;
+    let mut result = if is_negative { [255u8; 16] } else { [0u8; 16] };
+    for (d, s) in result.iter_mut().skip(16 - b.len()).zip(b) {
+        *d = *s;
+    }
+    result
+}
+
+/// Converts parquet RowGroup statistics (stored in
+/// [`RowGroupMetaData`]) into an arrow [`ArrayRef`]
+///
+/// For example, given a parquet file with 3 Row Groups, when asked for
+/// statistics for column "A" it will return a single array with 3 elements,
+///
+pub(crate) struct RowGroupStatisticsConverter<'a> {
+    field: &'a Field,
+}
+
+/// Extract a single min/max statistics from a [`ParquetStatistics`] object
+///
+/// * `$column_statistics` is the `ParquetStatistics` object
+/// * `$func is the function` (`min`/`max`) to call to get the value
+/// * `$bytes_func` is the function (`min_bytes`/`max_bytes`) to call to get the value as bytes
+/// * `$target_arrow_type` is the [`DataType`] of the target statistics
+macro_rules! get_statistic {
+    ($column_statistics:expr, $func:ident, $bytes_func:ident, $target_arrow_type:expr) => {{
+        if !$column_statistics.has_min_max_set() {
+            return None;
+        }
+        match $column_statistics {
+            ParquetStatistics::Boolean(s) => Some(ScalarValue::Boolean(Some(*s.$func()))),
+            ParquetStatistics::Int32(s) => {
+                match $target_arrow_type {
+                    // int32 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int32(Some(*s.$func()))),
+                }
+            }
+            ParquetStatistics::Int64(s) => {
+                match $target_arrow_type {
+                    // int64 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int64(Some(*s.$func()))),
+                }
+            }
+            // 96 bit ints not supported
+            ParquetStatistics::Int96(_) => None,
+            ParquetStatistics::Float(s) => Some(ScalarValue::Float32(Some(*s.$func()))),
+            ParquetStatistics::Double(s) => Some(ScalarValue::Float64(Some(*s.$func()))),
+            ParquetStatistics::ByteArray(s) => {
+                match $target_arrow_type {
+                    // decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => {
+                        let s = std::str::from_utf8(s.$bytes_func())
+                            .map(|s| s.to_string())
+                            .ok();
+                        Some(ScalarValue::Utf8(s))
+                    }
+                }
+            }
+            // type not supported yet
+            ParquetStatistics::FixedLenByteArray(s) => {
+                match $target_arrow_type {
+                    // just support the decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => None,
+                }
+            }
+        }
+    }};
+}
+
+#[derive(Debug, Clone, Copy)]
+enum MinMax {
+    Min,
+    Max,
+}
+
+impl<'a> RowGroupStatisticsConverter<'a> {

Review Comment:
   > columns by their name (in the arrow Schema)
   
   The problem is this is not how parquet addresses the columns, and trying to pretend otherwise is going to result in a number of peculiar bugs



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Extract parquet statistics to its own module, add tests [arrow-datafusion]

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on PR #8294:
URL: https://github.com/apache/arrow-datafusion/pull/8294#issuecomment-1830661634

   Thank you -- I plan to merge this tomorrow unless there are any other comments


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Extract parquet statistics to its own module, add tests [arrow-datafusion]

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on code in PR #8294:
URL: https://github.com/apache/arrow-datafusion/pull/8294#discussion_r1401959051


##########
datafusion/core/src/datasource/physical_plan/parquet/statistics.rs:
##########
@@ -0,0 +1,807 @@
+// 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.
+
+//! [`RowGroupStatisticsConverter`] tp converts parquet RowGroup statistics to arrow [`ArrayRef`].
+
+use arrow::{array::ArrayRef, datatypes::DataType};
+use arrow_array::new_empty_array;
+use arrow_schema::Field;
+use datafusion_common::{Result, ScalarValue};
+use parquet::file::{
+    metadata::RowGroupMetaData, statistics::Statistics as ParquetStatistics,
+};
+
+// 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(b))
+}
+
+// Copy from arrow-rs
+// https://github.com/apache/arrow-rs/blob/733b7e7fd1e8c43a404c3ce40ecf741d493c21b4/parquet/src/arrow/buffer/bit_util.rs#L55
+// Convert the byte slice to fixed length byte array with the length of 16
+fn sign_extend_be(b: &[u8]) -> [u8; 16] {
+    assert!(b.len() <= 16, "Array too large, expected less than 16");
+    let is_negative = (b[0] & 128u8) == 128u8;
+    let mut result = if is_negative { [255u8; 16] } else { [0u8; 16] };
+    for (d, s) in result.iter_mut().skip(16 - b.len()).zip(b) {
+        *d = *s;
+    }
+    result
+}
+
+/// Converts parquet RowGroup statistics (stored in
+/// [`RowGroupMetaData`]) into an arrow [`ArrayRef`]
+///
+/// For example, given a parquet file with 3 Row Groups, when asked for
+/// statistics for column "A" it will return a single array with 3 elements,
+///
+pub(crate) struct RowGroupStatisticsConverter<'a> {
+    field: &'a Field,
+}
+
+/// Extract a single min/max statistics from a [`ParquetStatistics`] object
+///
+/// * `$column_statistics` is the `ParquetStatistics` object
+/// * `$func is the function` (`min`/`max`) to call to get the value
+/// * `$bytes_func` is the function (`min_bytes`/`max_bytes`) to call to get the value as bytes
+/// * `$target_arrow_type` is the [`DataType`] of the target statistics
+macro_rules! get_statistic {
+    ($column_statistics:expr, $func:ident, $bytes_func:ident, $target_arrow_type:expr) => {{
+        if !$column_statistics.has_min_max_set() {
+            return None;
+        }
+        match $column_statistics {
+            ParquetStatistics::Boolean(s) => Some(ScalarValue::Boolean(Some(*s.$func()))),
+            ParquetStatistics::Int32(s) => {
+                match $target_arrow_type {
+                    // int32 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int32(Some(*s.$func()))),
+                }
+            }
+            ParquetStatistics::Int64(s) => {
+                match $target_arrow_type {
+                    // int64 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int64(Some(*s.$func()))),
+                }
+            }
+            // 96 bit ints not supported
+            ParquetStatistics::Int96(_) => None,
+            ParquetStatistics::Float(s) => Some(ScalarValue::Float32(Some(*s.$func()))),
+            ParquetStatistics::Double(s) => Some(ScalarValue::Float64(Some(*s.$func()))),
+            ParquetStatistics::ByteArray(s) => {
+                match $target_arrow_type {
+                    // decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => {
+                        let s = std::str::from_utf8(s.$bytes_func())
+                            .map(|s| s.to_string())
+                            .ok();
+                        Some(ScalarValue::Utf8(s))
+                    }
+                }
+            }
+            // type not supported yet
+            ParquetStatistics::FixedLenByteArray(s) => {
+                match $target_arrow_type {
+                    // just support the decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => None,
+                }
+            }
+        }
+    }};
+}
+
+#[derive(Debug, Clone, Copy)]
+enum MinMax {
+    Min,
+    Max,
+}
+
+impl<'a> RowGroupStatisticsConverter<'a> {
+    /// Create a new RowGoupStatisticsConverter suitable that can extract
+    /// statistics for the specified field
+    pub fn new(field: &'a Field) -> Self {
+        Self { field }
+    }
+
+    /// Returns the min value for the column into an array ref.
+    pub fn min<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(MinMax::Min, row_group_meta_data)
+    }
+
+    /// Returns the max value for the column into an array ref.
+    pub fn max<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(MinMax::Max, row_group_meta_data)
+    }
+
+    /// Extracts all min/max values for the column into an array ref.
+    fn min_max_impl<'b>(
+        &self,
+        mm: MinMax,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        let mut row_group_meta_data = row_group_meta_data.into_iter().peekable();
+
+        // if it is empty, return empty array
+        if row_group_meta_data.peek().is_none() {
+            return Ok(new_empty_array(self.field.data_type()));
+        }
+
+        let maybe_index = row_group_meta_data.peek().and_then(|rg_meta| {
+            rg_meta
+                .columns()
+                .iter()
+                .enumerate()
+                .find(|(_idx, c)| c.column_descr().name() == self.field.name())
+                .map(|(idx, _c)| idx)
+        });
+
+        // don't have this column, return an array of all NULLs
+        let Some(column_index) = maybe_index else {
+            let num_row_groups = row_group_meta_data.count();
+            let sv = ScalarValue::try_from(self.field.data_type())?;
+            return sv.to_array_of_size(num_row_groups);
+        };
+
+        let stats_iter = row_group_meta_data.map(move |row_group_meta_data| {
+            row_group_meta_data.column(column_index).statistics()
+        });
+
+        // this is the value to use when the statistics are not set
+        let null_value = ScalarValue::try_from(self.field.data_type())?;
+        match mm {
+            MinMax::Min => {
+                let values = stats_iter.map(|column_statistics| {
+                    column_statistics
+                        .and_then(|column_statistics| {
+                            get_statistic!(
+                                column_statistics,
+                                min,
+                                min_bytes,
+                                Some(self.field.data_type().clone())
+                            )
+                        })
+                        .unwrap_or_else(|| null_value.clone())
+                });
+                ScalarValue::iter_to_array(values)
+            }
+            MinMax::Max => {
+                let values = stats_iter.map(|column_statistics| {
+                    column_statistics
+                        .and_then(|column_statistics| {
+                            get_statistic!(
+                                column_statistics,
+                                max,
+                                max_bytes,
+                                Some(self.field.data_type().clone())
+                            )
+                        })
+                        .unwrap_or_else(|| null_value.clone())
+                });
+                ScalarValue::iter_to_array(values)
+            }
+        }
+    }
+}
+
+#[cfg(test)]
+mod test {
+    use super::*;
+    use arrow_array::{
+        BinaryArray, BooleanArray, Decimal128Array, Float32Array, Float64Array,
+        Int32Array, Int64Array, RecordBatch, StringArray, TimestampNanosecondArray,
+    };
+    use arrow_schema::SchemaRef;
+    use bytes::Bytes;
+    use datafusion_common::test_util::parquet_test_data;
+    use parquet::arrow::arrow_reader::ArrowReaderBuilder;
+    use parquet::arrow::arrow_writer::ArrowWriter;
+    use parquet::file::metadata::ParquetMetaData;
+    use parquet::file::properties::{EnabledStatistics, WriterProperties};
+    use std::path::PathBuf;
+    use std::sync::Arc;
+
+    // TODO error cases (with parquet statistics that are mismatched in expected type)
+
+    #[test]
+    fn roundtrip_empty() {
+        let empty_bool_array = new_empty_array(&DataType::Boolean);
+        Test {
+            input: empty_bool_array.clone(),
+            expected_min: empty_bool_array.clone(),
+            expected_max: empty_bool_array.clone(),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_bool() {
+        Test {
+            input: Arc::new(BooleanArray::from(vec![
+                // row group 1
+                Some(true),
+                None,
+                Some(true),
+                // row group 2
+                Some(true),
+                Some(false),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ])),
+            expected_min: Arc::new(BooleanArray::from(vec![
+                Some(true),
+                Some(false),
+                None,
+            ])),
+            expected_max: Arc::new(BooleanArray::from(vec![
+                Some(true),
+                Some(true),
+                None,
+            ])),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_int32() {
+        Test {
+            input: Arc::new(Int32Array::from(vec![
+                // row group 1
+                Some(1),
+                None,
+                Some(3),
+                // row group 2
+                Some(0),
+                Some(5),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ])),
+            expected_min: Arc::new(Int32Array::from(vec![Some(1), Some(0), None])),
+            expected_max: Arc::new(Int32Array::from(vec![Some(3), Some(5), None])),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_int64() {
+        Test {
+            input: Arc::new(Int64Array::from(vec![
+                // row group 1
+                Some(1),
+                None,
+                Some(3),
+                // row group 2
+                Some(0),
+                Some(5),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ])),
+            expected_min: Arc::new(Int64Array::from(vec![Some(1), Some(0), None])),
+            expected_max: Arc::new(Int64Array::from(vec![Some(3), Some(5), None])),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_f32() {
+        Test {
+            input: Arc::new(Float32Array::from(vec![
+                // 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: Arc::new(Float32Array::from(vec![Some(1.0), Some(-1.0), None])),
+            expected_max: Arc::new(Float32Array::from(vec![Some(3.0), Some(5.0), None])),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_f64() {
+        Test {
+            input: Arc::new(Float64Array::from(vec![
+                // 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: Arc::new(Float64Array::from(vec![Some(1.0), Some(-1.0), None])),
+            expected_max: Arc::new(Float64Array::from(vec![Some(3.0), Some(5.0), None])),
+        }
+        .run()
+    }
+
+    #[test]
+    #[should_panic(
+        expected = "Inconsistent types in ScalarValue::iter_to_array. Expected Int64, got TimestampNanosecond(NULL, None)"
+    )]
+    fn roundtrip_timestamp() {
+        Test {
+            input: Arc::new(TimestampNanosecondArray::from(vec![
+                // row group 1
+                Some(1),
+                None,
+                Some(3),
+                // row group 2
+                Some(9),
+                Some(5),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ])),
+            expected_min: Arc::new(TimestampNanosecondArray::from(vec![
+                Some(1),
+                Some(5),
+                None,
+            ])),
+            expected_max: Arc::new(TimestampNanosecondArray::from(vec![
+                Some(3),
+                Some(9),
+                None,
+            ])),
+        }
+        .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]
+    fn roundtrip_utf8() {
+        Test {
+            input: Arc::new(StringArray::from(vec![
+                // row group 1
+                Some("A"),
+                None,
+                Some("Q"),
+                // row group 2
+                Some("ZZ"),
+                Some("AA"),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ])),
+            expected_min: Arc::new(StringArray::from(vec![Some("A"), Some("AA"), None])),
+            expected_max: Arc::new(StringArray::from(vec![Some("Q"), Some("ZZ"), None])),
+        }
+        .run()
+    }
+
+    #[test]
+    #[should_panic(
+        expected = "Inconsistent types in ScalarValue::iter_to_array. Expected Utf8, got Binary(NULL)"
+    )]
+    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()
+    }
+
+    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);
+
+            for field in schema.fields() {
+                let converter = RowGroupStatisticsConverter::new(field);
+                let row_groups = metadata.row_groups();
+                let min = converter.min(row_groups).unwrap();
+                assert_eq!(
+                    &min,
+                    &expected_min,
+                    "Min. Statistics\n\n{}\n\n",
+                    DisplayStats(row_groups)
+                );
+
+                let max = converter.max(row_groups).unwrap();
+                assert_eq!(
+                    &max,
+                    &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<ParquetMetaData> {
+        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<ExpectedColumn>,
+    }
+
+    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 exepcted 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();
+
+            for rg in metadata.row_groups() {
+                println!(
+                    "Columns: {}",
+                    rg.columns()
+                        .iter()
+                        .map(|c| c.column_descr().name())
+                        .collect::<Vec<_>>()
+                        .join(", ")
+                );
+            }
+            println!("  row groups: {}", metadata.row_groups().len());
+
+            for expected_column in self.expected_columns {
+                let ExpectedColumn {
+                    name,
+                    expected_min,
+                    expected_max,
+                } = expected_column;
+
+                let field = arrow_schema
+                    .field_with_name(name)
+                    .expect("can't find field in schema");
+
+                let converter = RowGroupStatisticsConverter::new(field);
+                let actual_min = converter.min(metadata.row_groups()).unwrap();
+                assert_eq!(&expected_min, &actual_min, "column {name}");
+
+                let actual_max = converter.max(metadata.row_groups()).unwrap();
+                assert_eq!(&expected_max, &actual_max, "column {name}");
+            }
+        }
+    }
+
+    #[test]

Review Comment:
   I added a bunch of tests for reading statistics out of existing files to document what the current behavior is.
   
   Sadly, all of the example files in `parquet_testing` appear to have a single row group



##########
datafusion/core/src/datasource/physical_plan/parquet/statistics.rs:
##########
@@ -0,0 +1,807 @@
+// 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.
+
+//! [`RowGroupStatisticsConverter`] tp converts parquet RowGroup statistics to arrow [`ArrayRef`].
+
+use arrow::{array::ArrayRef, datatypes::DataType};
+use arrow_array::new_empty_array;
+use arrow_schema::Field;
+use datafusion_common::{Result, ScalarValue};
+use parquet::file::{
+    metadata::RowGroupMetaData, statistics::Statistics as ParquetStatistics,
+};
+
+// 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(b))
+}
+
+// Copy from arrow-rs
+// https://github.com/apache/arrow-rs/blob/733b7e7fd1e8c43a404c3ce40ecf741d493c21b4/parquet/src/arrow/buffer/bit_util.rs#L55
+// Convert the byte slice to fixed length byte array with the length of 16
+fn sign_extend_be(b: &[u8]) -> [u8; 16] {
+    assert!(b.len() <= 16, "Array too large, expected less than 16");
+    let is_negative = (b[0] & 128u8) == 128u8;
+    let mut result = if is_negative { [255u8; 16] } else { [0u8; 16] };
+    for (d, s) in result.iter_mut().skip(16 - b.len()).zip(b) {
+        *d = *s;
+    }
+    result
+}
+
+/// Converts parquet RowGroup statistics (stored in
+/// [`RowGroupMetaData`]) into an arrow [`ArrayRef`]
+///
+/// For example, given a parquet file with 3 Row Groups, when asked for
+/// statistics for column "A" it will return a single array with 3 elements,
+///
+pub(crate) struct RowGroupStatisticsConverter<'a> {
+    field: &'a Field,
+}
+
+/// Extract a single min/max statistics from a [`ParquetStatistics`] object
+///
+/// * `$column_statistics` is the `ParquetStatistics` object
+/// * `$func is the function` (`min`/`max`) to call to get the value
+/// * `$bytes_func` is the function (`min_bytes`/`max_bytes`) to call to get the value as bytes
+/// * `$target_arrow_type` is the [`DataType`] of the target statistics
+macro_rules! get_statistic {
+    ($column_statistics:expr, $func:ident, $bytes_func:ident, $target_arrow_type:expr) => {{
+        if !$column_statistics.has_min_max_set() {
+            return None;
+        }
+        match $column_statistics {
+            ParquetStatistics::Boolean(s) => Some(ScalarValue::Boolean(Some(*s.$func()))),
+            ParquetStatistics::Int32(s) => {
+                match $target_arrow_type {
+                    // int32 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int32(Some(*s.$func()))),
+                }
+            }
+            ParquetStatistics::Int64(s) => {
+                match $target_arrow_type {
+                    // int64 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int64(Some(*s.$func()))),
+                }
+            }
+            // 96 bit ints not supported
+            ParquetStatistics::Int96(_) => None,
+            ParquetStatistics::Float(s) => Some(ScalarValue::Float32(Some(*s.$func()))),
+            ParquetStatistics::Double(s) => Some(ScalarValue::Float64(Some(*s.$func()))),
+            ParquetStatistics::ByteArray(s) => {
+                match $target_arrow_type {
+                    // decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => {
+                        let s = std::str::from_utf8(s.$bytes_func())
+                            .map(|s| s.to_string())
+                            .ok();
+                        Some(ScalarValue::Utf8(s))
+                    }
+                }
+            }
+            // type not supported yet
+            ParquetStatistics::FixedLenByteArray(s) => {
+                match $target_arrow_type {
+                    // just support the decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => None,
+                }
+            }
+        }
+    }};
+}
+
+#[derive(Debug, Clone, Copy)]
+enum MinMax {
+    Min,
+    Max,
+}
+
+impl<'a> RowGroupStatisticsConverter<'a> {
+    /// Create a new RowGoupStatisticsConverter suitable that can extract
+    /// statistics for the specified field
+    pub fn new(field: &'a Field) -> Self {
+        Self { field }
+    }
+
+    /// Returns the min value for the column into an array ref.
+    pub fn min<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(MinMax::Min, row_group_meta_data)
+    }
+
+    /// Returns the max value for the column into an array ref.
+    pub fn max<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(MinMax::Max, row_group_meta_data)
+    }
+
+    /// Extracts all min/max values for the column into an array ref.
+    fn min_max_impl<'b>(
+        &self,
+        mm: MinMax,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        let mut row_group_meta_data = row_group_meta_data.into_iter().peekable();
+
+        // if it is empty, return empty array
+        if row_group_meta_data.peek().is_none() {
+            return Ok(new_empty_array(self.field.data_type()));
+        }
+
+        let maybe_index = row_group_meta_data.peek().and_then(|rg_meta| {
+            rg_meta
+                .columns()
+                .iter()
+                .enumerate()
+                .find(|(_idx, c)| c.column_descr().name() == self.field.name())
+                .map(|(idx, _c)| idx)
+        });
+
+        // don't have this column, return an array of all NULLs
+        let Some(column_index) = maybe_index else {
+            let num_row_groups = row_group_meta_data.count();
+            let sv = ScalarValue::try_from(self.field.data_type())?;
+            return sv.to_array_of_size(num_row_groups);
+        };
+
+        let stats_iter = row_group_meta_data.map(move |row_group_meta_data| {
+            row_group_meta_data.column(column_index).statistics()
+        });
+
+        // this is the value to use when the statistics are not set
+        let null_value = ScalarValue::try_from(self.field.data_type())?;
+        match mm {
+            MinMax::Min => {
+                let values = stats_iter.map(|column_statistics| {
+                    column_statistics
+                        .and_then(|column_statistics| {
+                            get_statistic!(
+                                column_statistics,
+                                min,
+                                min_bytes,
+                                Some(self.field.data_type().clone())
+                            )
+                        })
+                        .unwrap_or_else(|| null_value.clone())
+                });
+                ScalarValue::iter_to_array(values)
+            }
+            MinMax::Max => {
+                let values = stats_iter.map(|column_statistics| {
+                    column_statistics
+                        .and_then(|column_statistics| {
+                            get_statistic!(
+                                column_statistics,
+                                max,
+                                max_bytes,
+                                Some(self.field.data_type().clone())
+                            )
+                        })
+                        .unwrap_or_else(|| null_value.clone())
+                });
+                ScalarValue::iter_to_array(values)
+            }
+        }
+    }
+}
+
+#[cfg(test)]
+mod test {
+    use super::*;
+    use arrow_array::{
+        BinaryArray, BooleanArray, Decimal128Array, Float32Array, Float64Array,
+        Int32Array, Int64Array, RecordBatch, StringArray, TimestampNanosecondArray,
+    };
+    use arrow_schema::SchemaRef;
+    use bytes::Bytes;
+    use datafusion_common::test_util::parquet_test_data;
+    use parquet::arrow::arrow_reader::ArrowReaderBuilder;
+    use parquet::arrow::arrow_writer::ArrowWriter;
+    use parquet::file::metadata::ParquetMetaData;
+    use parquet::file::properties::{EnabledStatistics, WriterProperties};
+    use std::path::PathBuf;
+    use std::sync::Arc;
+
+    // TODO error cases (with parquet statistics that are mismatched in expected type)
+
+    #[test]
+    fn roundtrip_empty() {
+        let empty_bool_array = new_empty_array(&DataType::Boolean);
+        Test {
+            input: empty_bool_array.clone(),
+            expected_min: empty_bool_array.clone(),
+            expected_max: empty_bool_array.clone(),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_bool() {
+        Test {
+            input: Arc::new(BooleanArray::from(vec![
+                // row group 1
+                Some(true),
+                None,
+                Some(true),
+                // row group 2
+                Some(true),
+                Some(false),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ])),
+            expected_min: Arc::new(BooleanArray::from(vec![
+                Some(true),
+                Some(false),
+                None,
+            ])),
+            expected_max: Arc::new(BooleanArray::from(vec![
+                Some(true),
+                Some(true),
+                None,
+            ])),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_int32() {
+        Test {
+            input: Arc::new(Int32Array::from(vec![
+                // row group 1
+                Some(1),
+                None,
+                Some(3),
+                // row group 2
+                Some(0),
+                Some(5),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ])),
+            expected_min: Arc::new(Int32Array::from(vec![Some(1), Some(0), None])),
+            expected_max: Arc::new(Int32Array::from(vec![Some(3), Some(5), None])),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_int64() {
+        Test {
+            input: Arc::new(Int64Array::from(vec![
+                // row group 1
+                Some(1),
+                None,
+                Some(3),
+                // row group 2
+                Some(0),
+                Some(5),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ])),
+            expected_min: Arc::new(Int64Array::from(vec![Some(1), Some(0), None])),
+            expected_max: Arc::new(Int64Array::from(vec![Some(3), Some(5), None])),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_f32() {
+        Test {
+            input: Arc::new(Float32Array::from(vec![
+                // 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: Arc::new(Float32Array::from(vec![Some(1.0), Some(-1.0), None])),
+            expected_max: Arc::new(Float32Array::from(vec![Some(3.0), Some(5.0), None])),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_f64() {
+        Test {
+            input: Arc::new(Float64Array::from(vec![
+                // 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: Arc::new(Float64Array::from(vec![Some(1.0), Some(-1.0), None])),
+            expected_max: Arc::new(Float64Array::from(vec![Some(3.0), Some(5.0), None])),
+        }
+        .run()
+    }
+
+    #[test]
+    #[should_panic(
+        expected = "Inconsistent types in ScalarValue::iter_to_array. Expected Int64, got TimestampNanosecond(NULL, None)"
+    )]
+    fn roundtrip_timestamp() {

Review Comment:
   Tracked by https://github.com/apache/arrow-datafusion/issues/8295
   
   ```suggestion
       // Due to https://github.com/apache/arrow-datafusion/issues/8295
       fn roundtrip_timestamp() {
   ```



##########
datafusion/core/src/datasource/physical_plan/parquet/statistics.rs:
##########
@@ -0,0 +1,807 @@
+// 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.
+
+//! [`RowGroupStatisticsConverter`] tp converts parquet RowGroup statistics to arrow [`ArrayRef`].
+
+use arrow::{array::ArrayRef, datatypes::DataType};
+use arrow_array::new_empty_array;
+use arrow_schema::Field;
+use datafusion_common::{Result, ScalarValue};
+use parquet::file::{
+    metadata::RowGroupMetaData, statistics::Statistics as ParquetStatistics,
+};
+
+// 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(b))
+}
+
+// Copy from arrow-rs
+// https://github.com/apache/arrow-rs/blob/733b7e7fd1e8c43a404c3ce40ecf741d493c21b4/parquet/src/arrow/buffer/bit_util.rs#L55
+// Convert the byte slice to fixed length byte array with the length of 16
+fn sign_extend_be(b: &[u8]) -> [u8; 16] {
+    assert!(b.len() <= 16, "Array too large, expected less than 16");
+    let is_negative = (b[0] & 128u8) == 128u8;
+    let mut result = if is_negative { [255u8; 16] } else { [0u8; 16] };
+    for (d, s) in result.iter_mut().skip(16 - b.len()).zip(b) {
+        *d = *s;
+    }
+    result
+}
+
+/// Converts parquet RowGroup statistics (stored in
+/// [`RowGroupMetaData`]) into an arrow [`ArrayRef`]
+///
+/// For example, given a parquet file with 3 Row Groups, when asked for
+/// statistics for column "A" it will return a single array with 3 elements,
+///
+pub(crate) struct RowGroupStatisticsConverter<'a> {
+    field: &'a Field,
+}
+
+/// Extract a single min/max statistics from a [`ParquetStatistics`] object
+///
+/// * `$column_statistics` is the `ParquetStatistics` object
+/// * `$func is the function` (`min`/`max`) to call to get the value
+/// * `$bytes_func` is the function (`min_bytes`/`max_bytes`) to call to get the value as bytes
+/// * `$target_arrow_type` is the [`DataType`] of the target statistics
+macro_rules! get_statistic {
+    ($column_statistics:expr, $func:ident, $bytes_func:ident, $target_arrow_type:expr) => {{
+        if !$column_statistics.has_min_max_set() {
+            return None;
+        }
+        match $column_statistics {
+            ParquetStatistics::Boolean(s) => Some(ScalarValue::Boolean(Some(*s.$func()))),
+            ParquetStatistics::Int32(s) => {
+                match $target_arrow_type {
+                    // int32 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int32(Some(*s.$func()))),
+                }
+            }
+            ParquetStatistics::Int64(s) => {
+                match $target_arrow_type {
+                    // int64 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int64(Some(*s.$func()))),
+                }
+            }
+            // 96 bit ints not supported
+            ParquetStatistics::Int96(_) => None,
+            ParquetStatistics::Float(s) => Some(ScalarValue::Float32(Some(*s.$func()))),
+            ParquetStatistics::Double(s) => Some(ScalarValue::Float64(Some(*s.$func()))),
+            ParquetStatistics::ByteArray(s) => {
+                match $target_arrow_type {
+                    // decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => {
+                        let s = std::str::from_utf8(s.$bytes_func())
+                            .map(|s| s.to_string())
+                            .ok();
+                        Some(ScalarValue::Utf8(s))
+                    }
+                }
+            }
+            // type not supported yet
+            ParquetStatistics::FixedLenByteArray(s) => {
+                match $target_arrow_type {
+                    // just support the decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => None,
+                }
+            }
+        }
+    }};
+}
+
+#[derive(Debug, Clone, Copy)]
+enum MinMax {
+    Min,
+    Max,
+}
+
+impl<'a> RowGroupStatisticsConverter<'a> {

Review Comment:
   @tustvold  I would be interested in hearing your thoughts or suggestions on how to improve this interface (is this per field wrapping reasonable?)



##########
datafusion/core/src/datasource/physical_plan/parquet/statistics.rs:
##########
@@ -0,0 +1,807 @@
+// 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.
+
+//! [`RowGroupStatisticsConverter`] tp converts parquet RowGroup statistics to arrow [`ArrayRef`].
+
+use arrow::{array::ArrayRef, datatypes::DataType};
+use arrow_array::new_empty_array;
+use arrow_schema::Field;
+use datafusion_common::{Result, ScalarValue};
+use parquet::file::{
+    metadata::RowGroupMetaData, statistics::Statistics as ParquetStatistics,
+};
+
+// 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(b))
+}
+
+// Copy from arrow-rs
+// https://github.com/apache/arrow-rs/blob/733b7e7fd1e8c43a404c3ce40ecf741d493c21b4/parquet/src/arrow/buffer/bit_util.rs#L55
+// Convert the byte slice to fixed length byte array with the length of 16
+fn sign_extend_be(b: &[u8]) -> [u8; 16] {
+    assert!(b.len() <= 16, "Array too large, expected less than 16");
+    let is_negative = (b[0] & 128u8) == 128u8;
+    let mut result = if is_negative { [255u8; 16] } else { [0u8; 16] };
+    for (d, s) in result.iter_mut().skip(16 - b.len()).zip(b) {
+        *d = *s;
+    }
+    result
+}
+
+/// Converts parquet RowGroup statistics (stored in
+/// [`RowGroupMetaData`]) into an arrow [`ArrayRef`]
+///
+/// For example, given a parquet file with 3 Row Groups, when asked for
+/// statistics for column "A" it will return a single array with 3 elements,
+///
+pub(crate) struct RowGroupStatisticsConverter<'a> {
+    field: &'a Field,
+}
+
+/// Extract a single min/max statistics from a [`ParquetStatistics`] object
+///
+/// * `$column_statistics` is the `ParquetStatistics` object
+/// * `$func is the function` (`min`/`max`) to call to get the value
+/// * `$bytes_func` is the function (`min_bytes`/`max_bytes`) to call to get the value as bytes
+/// * `$target_arrow_type` is the [`DataType`] of the target statistics
+macro_rules! get_statistic {
+    ($column_statistics:expr, $func:ident, $bytes_func:ident, $target_arrow_type:expr) => {{
+        if !$column_statistics.has_min_max_set() {
+            return None;
+        }
+        match $column_statistics {
+            ParquetStatistics::Boolean(s) => Some(ScalarValue::Boolean(Some(*s.$func()))),
+            ParquetStatistics::Int32(s) => {
+                match $target_arrow_type {
+                    // int32 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int32(Some(*s.$func()))),
+                }
+            }
+            ParquetStatistics::Int64(s) => {
+                match $target_arrow_type {
+                    // int64 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int64(Some(*s.$func()))),
+                }
+            }
+            // 96 bit ints not supported
+            ParquetStatistics::Int96(_) => None,
+            ParquetStatistics::Float(s) => Some(ScalarValue::Float32(Some(*s.$func()))),
+            ParquetStatistics::Double(s) => Some(ScalarValue::Float64(Some(*s.$func()))),
+            ParquetStatistics::ByteArray(s) => {
+                match $target_arrow_type {
+                    // decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => {
+                        let s = std::str::from_utf8(s.$bytes_func())
+                            .map(|s| s.to_string())
+                            .ok();
+                        Some(ScalarValue::Utf8(s))
+                    }
+                }
+            }
+            // type not supported yet
+            ParquetStatistics::FixedLenByteArray(s) => {
+                match $target_arrow_type {
+                    // just support the decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => None,
+                }
+            }
+        }
+    }};
+}
+
+#[derive(Debug, Clone, Copy)]
+enum MinMax {
+    Min,
+    Max,
+}
+
+impl<'a> RowGroupStatisticsConverter<'a> {
+    /// Create a new RowGoupStatisticsConverter suitable that can extract
+    /// statistics for the specified field
+    pub fn new(field: &'a Field) -> Self {
+        Self { field }
+    }
+
+    /// Returns the min value for the column into an array ref.
+    pub fn min<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(MinMax::Min, row_group_meta_data)
+    }
+
+    /// Returns the max value for the column into an array ref.
+    pub fn max<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(MinMax::Max, row_group_meta_data)
+    }
+
+    /// Extracts all min/max values for the column into an array ref.
+    fn min_max_impl<'b>(
+        &self,
+        mm: MinMax,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        let mut row_group_meta_data = row_group_meta_data.into_iter().peekable();
+
+        // if it is empty, return empty array
+        if row_group_meta_data.peek().is_none() {
+            return Ok(new_empty_array(self.field.data_type()));
+        }
+
+        let maybe_index = row_group_meta_data.peek().and_then(|rg_meta| {
+            rg_meta
+                .columns()
+                .iter()
+                .enumerate()
+                .find(|(_idx, c)| c.column_descr().name() == self.field.name())
+                .map(|(idx, _c)| idx)
+        });
+
+        // don't have this column, return an array of all NULLs
+        let Some(column_index) = maybe_index else {
+            let num_row_groups = row_group_meta_data.count();
+            let sv = ScalarValue::try_from(self.field.data_type())?;
+            return sv.to_array_of_size(num_row_groups);
+        };
+
+        let stats_iter = row_group_meta_data.map(move |row_group_meta_data| {
+            row_group_meta_data.column(column_index).statistics()
+        });
+
+        // this is the value to use when the statistics are not set
+        let null_value = ScalarValue::try_from(self.field.data_type())?;
+        match mm {
+            MinMax::Min => {
+                let values = stats_iter.map(|column_statistics| {
+                    column_statistics
+                        .and_then(|column_statistics| {
+                            get_statistic!(
+                                column_statistics,
+                                min,
+                                min_bytes,
+                                Some(self.field.data_type().clone())
+                            )
+                        })
+                        .unwrap_or_else(|| null_value.clone())
+                });
+                ScalarValue::iter_to_array(values)
+            }
+            MinMax::Max => {
+                let values = stats_iter.map(|column_statistics| {
+                    column_statistics
+                        .and_then(|column_statistics| {
+                            get_statistic!(
+                                column_statistics,
+                                max,
+                                max_bytes,
+                                Some(self.field.data_type().clone())
+                            )
+                        })
+                        .unwrap_or_else(|| null_value.clone())
+                });
+                ScalarValue::iter_to_array(values)
+            }
+        }
+    }
+}
+
+#[cfg(test)]
+mod test {
+    use super::*;
+    use arrow_array::{
+        BinaryArray, BooleanArray, Decimal128Array, Float32Array, Float64Array,
+        Int32Array, Int64Array, RecordBatch, StringArray, TimestampNanosecondArray,
+    };
+    use arrow_schema::SchemaRef;
+    use bytes::Bytes;
+    use datafusion_common::test_util::parquet_test_data;
+    use parquet::arrow::arrow_reader::ArrowReaderBuilder;
+    use parquet::arrow::arrow_writer::ArrowWriter;
+    use parquet::file::metadata::ParquetMetaData;
+    use parquet::file::properties::{EnabledStatistics, WriterProperties};
+    use std::path::PathBuf;
+    use std::sync::Arc;
+
+    // TODO error cases (with parquet statistics that are mismatched in expected type)
+
+    #[test]
+    fn roundtrip_empty() {
+        let empty_bool_array = new_empty_array(&DataType::Boolean);
+        Test {
+            input: empty_bool_array.clone(),
+            expected_min: empty_bool_array.clone(),
+            expected_max: empty_bool_array.clone(),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_bool() {
+        Test {
+            input: Arc::new(BooleanArray::from(vec![
+                // row group 1
+                Some(true),
+                None,
+                Some(true),
+                // row group 2
+                Some(true),
+                Some(false),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ])),
+            expected_min: Arc::new(BooleanArray::from(vec![
+                Some(true),
+                Some(false),
+                None,
+            ])),
+            expected_max: Arc::new(BooleanArray::from(vec![
+                Some(true),
+                Some(true),
+                None,
+            ])),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_int32() {
+        Test {
+            input: Arc::new(Int32Array::from(vec![
+                // row group 1
+                Some(1),
+                None,
+                Some(3),
+                // row group 2
+                Some(0),
+                Some(5),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ])),
+            expected_min: Arc::new(Int32Array::from(vec![Some(1), Some(0), None])),
+            expected_max: Arc::new(Int32Array::from(vec![Some(3), Some(5), None])),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_int64() {
+        Test {
+            input: Arc::new(Int64Array::from(vec![
+                // row group 1
+                Some(1),
+                None,
+                Some(3),
+                // row group 2
+                Some(0),
+                Some(5),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ])),
+            expected_min: Arc::new(Int64Array::from(vec![Some(1), Some(0), None])),
+            expected_max: Arc::new(Int64Array::from(vec![Some(3), Some(5), None])),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_f32() {
+        Test {
+            input: Arc::new(Float32Array::from(vec![
+                // 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: Arc::new(Float32Array::from(vec![Some(1.0), Some(-1.0), None])),
+            expected_max: Arc::new(Float32Array::from(vec![Some(3.0), Some(5.0), None])),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_f64() {
+        Test {
+            input: Arc::new(Float64Array::from(vec![
+                // 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: Arc::new(Float64Array::from(vec![Some(1.0), Some(-1.0), None])),
+            expected_max: Arc::new(Float64Array::from(vec![Some(3.0), Some(5.0), None])),
+        }
+        .run()
+    }
+
+    #[test]
+    #[should_panic(
+        expected = "Inconsistent types in ScalarValue::iter_to_array. Expected Int64, got TimestampNanosecond(NULL, None)"
+    )]
+    fn roundtrip_timestamp() {
+        Test {
+            input: Arc::new(TimestampNanosecondArray::from(vec![
+                // row group 1
+                Some(1),
+                None,
+                Some(3),
+                // row group 2
+                Some(9),
+                Some(5),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ])),
+            expected_min: Arc::new(TimestampNanosecondArray::from(vec![
+                Some(1),
+                Some(5),
+                None,
+            ])),
+            expected_max: Arc::new(TimestampNanosecondArray::from(vec![
+                Some(3),
+                Some(9),
+                None,
+            ])),
+        }
+        .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]
+    fn roundtrip_utf8() {
+        Test {
+            input: Arc::new(StringArray::from(vec![
+                // row group 1
+                Some("A"),
+                None,
+                Some("Q"),
+                // row group 2
+                Some("ZZ"),
+                Some("AA"),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ])),
+            expected_min: Arc::new(StringArray::from(vec![Some("A"), Some("AA"), None])),
+            expected_max: Arc::new(StringArray::from(vec![Some("Q"), Some("ZZ"), None])),
+        }
+        .run()
+    }
+
+    #[test]
+    #[should_panic(
+        expected = "Inconsistent types in ScalarValue::iter_to_array. Expected Utf8, got Binary(NULL)"
+    )]
+    fn roundtrip_binary() {

Review Comment:
   ```suggestion
       // Due to https://github.com/apache/arrow-datafusion/issues/8295
       fn roundtrip_binary() {
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Extract parquet statistics to its own module, add tests [arrow-datafusion]

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on PR #8294:
URL: https://github.com/apache/arrow-datafusion/pull/8294#issuecomment-1830073857

   I incorporated @tustvold 's PR here: https://github.com/alamb/arrow-datafusion/pull/16


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Extract parquet statistics to its own module, add tests [arrow-datafusion]

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on PR #8294:
URL: https://github.com/apache/arrow-datafusion/pull/8294#issuecomment-1828170161

   I spoke with @tustvold and we came up with the following plan:
   
   
   Plans:
   1. Change this PR to take iterator of [`ColumnMetadata`](https://docs.rs/parquet/latest/parquet/format/struct.ColumnMetaData.html) and a target arrow `DataType` so we can consolidate the "map arrow column -> parquet column" logic (which is incorrect for `struct`s, as pointed out above) 
   1. (as a follow on PR): Add a test for reading/writing struct arrays with statistics (and likely file a ticket when this turns out to be wrong). The test should also ensure that reading a scalar field that appears *after* the struct field gets the correct values.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Extract parquet statistics to its own module, add tests [arrow-datafusion]

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on code in PR #8294:
URL: https://github.com/apache/arrow-datafusion/pull/8294#discussion_r1406743880


##########
datafusion/core/src/datasource/physical_plan/parquet/statistics.rs:
##########
@@ -0,0 +1,925 @@
+// 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.
+
+//! [`RowGroupStatisticsConverter`] tp converts parquet RowGroup statistics to arrow [`ArrayRef`].
+
+use arrow::{array::ArrayRef, datatypes::DataType};
+use arrow_array::new_empty_array;
+use arrow_schema::Field;
+use datafusion_common::{Result, ScalarValue};
+use parquet::file::{
+    metadata::RowGroupMetaData, statistics::Statistics as ParquetStatistics,
+};
+
+// 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(b))
+}
+
+// Copy from arrow-rs
+// https://github.com/apache/arrow-rs/blob/733b7e7fd1e8c43a404c3ce40ecf741d493c21b4/parquet/src/arrow/buffer/bit_util.rs#L55
+// Convert the byte slice to fixed length byte array with the length of 16
+fn sign_extend_be(b: &[u8]) -> [u8; 16] {
+    assert!(b.len() <= 16, "Array too large, expected less than 16");
+    let is_negative = (b[0] & 128u8) == 128u8;
+    let mut result = if is_negative { [255u8; 16] } else { [0u8; 16] };
+    for (d, s) in result.iter_mut().skip(16 - b.len()).zip(b) {
+        *d = *s;
+    }
+    result
+}
+
+/// Extract a single min/max statistics from a [`ParquetStatistics`] object
+///
+/// * `$column_statistics` is the `ParquetStatistics` object
+/// * `$func is the function` (`min`/`max`) to call to get the value
+/// * `$bytes_func` is the function (`min_bytes`/`max_bytes`) to call to get the value as bytes
+/// * `$target_arrow_type` is the [`DataType`] of the target statistics
+macro_rules! get_statistic {
+    ($column_statistics:expr, $func:ident, $bytes_func:ident, $target_arrow_type:expr) => {{
+        if !$column_statistics.has_min_max_set() {
+            return None;
+        }
+        match $column_statistics {
+            ParquetStatistics::Boolean(s) => Some(ScalarValue::Boolean(Some(*s.$func()))),
+            ParquetStatistics::Int32(s) => {
+                match $target_arrow_type {
+                    // int32 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int32(Some(*s.$func()))),
+                }
+            }
+            ParquetStatistics::Int64(s) => {
+                match $target_arrow_type {
+                    // int64 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int64(Some(*s.$func()))),
+                }
+            }
+            // 96 bit ints not supported
+            ParquetStatistics::Int96(_) => None,
+            ParquetStatistics::Float(s) => Some(ScalarValue::Float32(Some(*s.$func()))),
+            ParquetStatistics::Double(s) => Some(ScalarValue::Float64(Some(*s.$func()))),
+            ParquetStatistics::ByteArray(s) => {
+                match $target_arrow_type {
+                    // decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => {
+                        let s = std::str::from_utf8(s.$bytes_func())
+                            .map(|s| s.to_string())
+                            .ok();
+                        Some(ScalarValue::Utf8(s))
+                    }
+                }
+            }
+            // type not supported yet
+            ParquetStatistics::FixedLenByteArray(s) => {
+                match $target_arrow_type {
+                    // just support the decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => None,
+                }
+            }
+        }
+    }};
+}
+
+#[derive(Debug, Clone, Copy)]
+enum Statistic {
+    Min,
+    Max,
+}
+
+/// Extracts statistics for a single leaf column from [`RowGroupMetaData`] as an
+/// arrow [`ArrayRef`]
+///
+/// For example, given a parquet file with 3 Row Groups, when asked for
+/// statistics for column "A" it will return a single array with 3 elements.
+pub(crate) struct RowGroupStatisticsConverter<'a> {
+    field: &'a Field,
+}
+
+impl<'a> RowGroupStatisticsConverter<'a> {
+    /// Create a new RowGoupStatisticsConverter for extracting the named column
+    /// in the schema.
+    pub fn try_new(schema: &'a arrow_schema::Schema, column_name: &str) -> Result<Self> {
+        let field = schema.field_with_name(column_name)?;
+        Ok(Self { field })
+    }
+
+    /// Returns the min value for the column into an array ref.
+    pub fn min<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(Statistic::Min, row_group_meta_data)
+    }
+
+    /// Returns the max value for the column into an array ref.
+    pub fn max<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(Statistic::Max, row_group_meta_data)
+    }
+
+    /// Extracts all min/max values for the column into an array ref.
+    fn min_max_impl<'b>(
+        &self,
+        mm: Statistic,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        let mut row_group_meta_data = row_group_meta_data.into_iter().peekable();
+
+        // if it is empty, return empty array
+        if row_group_meta_data.peek().is_none() {
+            return Ok(new_empty_array(self.field.data_type()));
+        }
+
+        let maybe_index = row_group_meta_data.peek().and_then(|rg_meta| {
+            rg_meta
+                .columns()
+                .iter()
+                .enumerate()
+                .find(|(_idx, c)| c.column_descr().name() == self.field.name())
+                .map(|(idx, _c)| idx)
+        });
+
+        // don't have this column, return an array of all NULLs
+        let Some(column_index) = maybe_index else {
+            let num_row_groups = row_group_meta_data.count();
+            let sv = ScalarValue::try_from(self.field.data_type())?;
+            return sv.to_array_of_size(num_row_groups);
+        };
+
+        let stats_iter = row_group_meta_data.map(move |row_group_meta_data| {
+            row_group_meta_data.column(column_index).statistics()
+        });
+
+        // this is the value to use when the statistics are not set
+        let null_value = ScalarValue::try_from(self.field.data_type())?;
+        match mm {
+            Statistic::Min => {
+                let values = stats_iter.map(|column_statistics| {
+                    column_statistics
+                        .and_then(|column_statistics| {
+                            get_statistic!(
+                                column_statistics,
+                                min,
+                                min_bytes,
+                                Some(self.field.data_type().clone())
+                            )
+                        })
+                        .unwrap_or_else(|| null_value.clone())
+                });
+                ScalarValue::iter_to_array(values)
+            }
+            Statistic::Max => {
+                let values = stats_iter.map(|column_statistics| {
+                    column_statistics
+                        .and_then(|column_statistics| {
+                            get_statistic!(
+                                column_statistics,
+                                max,
+                                max_bytes,
+                                Some(self.field.data_type().clone())
+                            )
+                        })
+                        .unwrap_or_else(|| null_value.clone())
+                });
+                ScalarValue::iter_to_array(values)
+            }
+        }
+    }
+}
+
+#[cfg(test)]
+mod test {
+    use super::*;
+    use arrow_array::{
+        new_null_array, Array, BinaryArray, BooleanArray, Decimal128Array, Float32Array,
+        Float64Array, Int32Array, Int64Array, RecordBatch, StringArray, StructArray,
+        TimestampNanosecondArray,
+    };
+    use arrow_schema::SchemaRef;
+    use bytes::Bytes;
+    use datafusion_common::test_util::parquet_test_data;
+    use parquet::arrow::arrow_reader::ArrowReaderBuilder;
+    use parquet::arrow::arrow_writer::ArrowWriter;
+    use parquet::file::metadata::ParquetMetaData;
+    use parquet::file::properties::{EnabledStatistics, WriterProperties};
+    use std::path::PathBuf;
+    use std::sync::Arc;
+
+    // TODO error cases (with parquet statistics that are mismatched in expected type)
+
+    #[test]
+    fn roundtrip_empty() {
+        let empty_bool_array = new_empty_array(&DataType::Boolean);
+        Test {
+            input: empty_bool_array.clone(),
+            expected_min: empty_bool_array.clone(),
+            expected_max: empty_bool_array.clone(),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_bool() {
+        Test {
+            input: bool_array([
+                // row group 1
+                Some(true),
+                None,
+                Some(true),
+                // row group 2
+                Some(true),
+                Some(false),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ]),
+            expected_min: bool_array([Some(true), Some(false), None]),
+            expected_max: bool_array([Some(true), Some(true), None]),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_int32() {
+        Test {
+            input: i32_array([
+                // row group 1
+                Some(1),
+                None,
+                Some(3),
+                // row group 2
+                Some(0),
+                Some(5),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ]),
+            expected_min: i32_array([Some(1), Some(0), None]),
+            expected_max: i32_array([Some(3), Some(5), None]),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_int64() {
+        Test {
+            input: i64_array([
+                // row group 1
+                Some(1),
+                None,
+                Some(3),
+                // row group 2
+                Some(0),
+                Some(5),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ]),
+            expected_min: i64_array([Some(1), Some(0), None]),
+            expected_max: i64_array(vec![Some(3), Some(5), None]),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_f32() {
+        Test {
+            input: f32_array([
+                // row group 1
+                Some(1.0),
+                None,
+                Some(3.0),
+                // row group 2
+                Some(-1.0),
+                Some(5.0),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ]),
+            expected_min: f32_array([Some(1.0), Some(-1.0), None]),
+            expected_max: f32_array([Some(3.0), Some(5.0), None]),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_f64() {
+        Test {
+            input: f64_array([
+                // row group 1
+                Some(1.0),
+                None,
+                Some(3.0),
+                // row group 2
+                Some(-1.0),
+                Some(5.0),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ]),
+            expected_min: f64_array([Some(1.0), Some(-1.0), None]),
+            expected_max: f64_array([Some(3.0), Some(5.0), None]),
+        }
+        .run()
+    }
+
+    #[test]
+    #[should_panic(
+        expected = "Inconsistent types in ScalarValue::iter_to_array. Expected Int64, got TimestampNanosecond(NULL, None)"
+    )]
+    // Due to https://github.com/apache/arrow-datafusion/issues/8295
+    fn roundtrip_timestamp() {
+        Test {
+            input: timestamp_array([
+                // row group 1
+                Some(1),
+                None,
+                Some(3),
+                // row group 2
+                Some(9),
+                Some(5),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ]),
+            expected_min: timestamp_array([Some(1), Some(5), None]),
+            expected_max: timestamp_array([Some(3), Some(9), None]),
+        }
+        .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]
+    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/arrow-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]
+    #[should_panic(
+        expected = "Inconsistent types in ScalarValue::iter_to_array. Expected Utf8, got Binary(NULL)"
+    )]
+    // Due to https://github.com/apache/arrow-datafusion/issues/8295
+    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 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)]);
+
+        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);
+
+        // read the int_col statistics
+        let (_idx, field) = schema.column_with_name("int_col").unwrap();
+
+        let converter =
+            RowGroupStatisticsConverter::try_new(&schema, field.name()).unwrap();
+        let row_groups = metadata.row_groups();
+        let min = converter.min(row_groups).unwrap();
+        assert_eq!(
+            &min,
+            &expected_min,
+            "Min. Statistics\n\n{}\n\n",
+            DisplayStats(row_groups)
+        );
+
+        let max = converter.max(row_groups).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: i32_array([Some(0)]),
+                expected_max: i32_array([Some(9)]),
+            })
+            .with_column(ExpectedColumn {
+                name: "smallint_col",
+                expected_min: i32_array([Some(0)]),
+                expected_max: i32_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_array([None]),
+                expected_max: timestamp_array([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);
+
+            for field in schema.fields() {
+                let converter =
+                    RowGroupStatisticsConverter::try_new(&schema, field.name()).unwrap();
+                let row_groups = metadata.row_groups();
+                let min = converter.min(row_groups).unwrap();
+                assert_eq!(
+                    &min,
+                    &expected_min,
+                    "Min. Statistics\n\n{}\n\n",
+                    DisplayStats(row_groups)
+                );
+
+                let max = converter.max(row_groups).unwrap();
+                assert_eq!(
+                    &max,
+                    &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<ParquetMetaData> {
+        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<ExpectedColumn>,
+    }
+
+    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 exepcted 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();
+
+            for expected_column in self.expected_columns {
+                let ExpectedColumn {
+                    name,
+                    expected_min,
+                    expected_max,
+                } = expected_column;
+
+                let converter = RowGroupStatisticsConverter::try_new(arrow_schema, name)
+                    .expect("can't find field in schema");
+
+                let actual_min = converter.min(metadata.row_groups()).unwrap();
+                assert_eq!(&expected_min, &actual_min, "column {name}");
+
+                let actual_max = converter.max(metadata.row_groups()).unwrap();
+                assert_eq!(&expected_max, &actual_max, "column {name}");
+            }
+        }
+    }
+
+    fn bool_array(input: impl IntoIterator<Item = Option<bool>>) -> ArrayRef {
+        let array: BooleanArray = input.into_iter().collect();
+        Arc::new(array)
+    }
+
+    fn i32_array(input: impl IntoIterator<Item = Option<i32>>) -> ArrayRef {
+        let array: Int32Array = input.into_iter().collect();
+        Arc::new(array)
+    }
+
+    fn i64_array(input: impl IntoIterator<Item = Option<i64>>) -> ArrayRef {
+        let array: Int64Array = input.into_iter().collect();
+        Arc::new(array)
+    }
+
+    fn f32_array(input: impl IntoIterator<Item = Option<f32>>) -> ArrayRef {
+        let array: Float32Array = input.into_iter().collect();
+        Arc::new(array)
+    }
+
+    fn f64_array(input: impl IntoIterator<Item = Option<f64>>) -> ArrayRef {
+        let array: Float64Array = input.into_iter().collect();
+        Arc::new(array)
+    }
+
+    fn timestamp_array(input: impl IntoIterator<Item = Option<i64>>) -> ArrayRef {
+        let array: TimestampNanosecondArray = input.into_iter().collect();
+        Arc::new(array)
+    }
+
+    fn utf8_array<'a>(input: impl IntoIterator<Item = Option<&'a str>>) -> 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 "b" and "i" with the specified values
+    fn struct_array(input: Vec<(Option<bool>, Option<i32>)>) -> 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("b", DataType::Boolean, nullable)),
+                Arc::new(boolean) as ArrayRef,
+            ),
+            (
+                Arc::new(Field::new("i", DataType::Int32, nullable)),

Review Comment:
   ```suggestion
                   Arc::new(Field::new("int_col", DataType::Int32, nullable)),
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Extract parquet statistics to its own module, add tests [arrow-datafusion]

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on PR #8294:
URL: https://github.com/apache/arrow-datafusion/pull/8294#issuecomment-1823239124

   FYI @viirya @liukun4515  and @Ted-Jiang 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Extract parquet statistics to its own module, add tests [arrow-datafusion]

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on code in PR #8294:
URL: https://github.com/apache/arrow-datafusion/pull/8294#discussion_r1406836605


##########
datafusion/core/src/datasource/physical_plan/parquet/statistics.rs:
##########
@@ -0,0 +1,925 @@
+// 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.
+
+//! [`RowGroupStatisticsConverter`] tp converts parquet RowGroup statistics to arrow [`ArrayRef`].
+
+use arrow::{array::ArrayRef, datatypes::DataType};
+use arrow_array::new_empty_array;
+use arrow_schema::Field;
+use datafusion_common::{Result, ScalarValue};
+use parquet::file::{
+    metadata::RowGroupMetaData, statistics::Statistics as ParquetStatistics,
+};
+
+// 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(b))
+}
+
+// Copy from arrow-rs
+// https://github.com/apache/arrow-rs/blob/733b7e7fd1e8c43a404c3ce40ecf741d493c21b4/parquet/src/arrow/buffer/bit_util.rs#L55
+// Convert the byte slice to fixed length byte array with the length of 16
+fn sign_extend_be(b: &[u8]) -> [u8; 16] {
+    assert!(b.len() <= 16, "Array too large, expected less than 16");
+    let is_negative = (b[0] & 128u8) == 128u8;
+    let mut result = if is_negative { [255u8; 16] } else { [0u8; 16] };
+    for (d, s) in result.iter_mut().skip(16 - b.len()).zip(b) {
+        *d = *s;
+    }
+    result
+}
+
+/// Extract a single min/max statistics from a [`ParquetStatistics`] object
+///
+/// * `$column_statistics` is the `ParquetStatistics` object
+/// * `$func is the function` (`min`/`max`) to call to get the value
+/// * `$bytes_func` is the function (`min_bytes`/`max_bytes`) to call to get the value as bytes
+/// * `$target_arrow_type` is the [`DataType`] of the target statistics
+macro_rules! get_statistic {
+    ($column_statistics:expr, $func:ident, $bytes_func:ident, $target_arrow_type:expr) => {{
+        if !$column_statistics.has_min_max_set() {
+            return None;
+        }
+        match $column_statistics {
+            ParquetStatistics::Boolean(s) => Some(ScalarValue::Boolean(Some(*s.$func()))),
+            ParquetStatistics::Int32(s) => {
+                match $target_arrow_type {
+                    // int32 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int32(Some(*s.$func()))),
+                }
+            }
+            ParquetStatistics::Int64(s) => {
+                match $target_arrow_type {
+                    // int64 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int64(Some(*s.$func()))),
+                }
+            }
+            // 96 bit ints not supported
+            ParquetStatistics::Int96(_) => None,
+            ParquetStatistics::Float(s) => Some(ScalarValue::Float32(Some(*s.$func()))),
+            ParquetStatistics::Double(s) => Some(ScalarValue::Float64(Some(*s.$func()))),
+            ParquetStatistics::ByteArray(s) => {
+                match $target_arrow_type {
+                    // decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => {
+                        let s = std::str::from_utf8(s.$bytes_func())
+                            .map(|s| s.to_string())
+                            .ok();
+                        Some(ScalarValue::Utf8(s))
+                    }
+                }
+            }
+            // type not supported yet
+            ParquetStatistics::FixedLenByteArray(s) => {
+                match $target_arrow_type {
+                    // just support the decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => None,
+                }
+            }
+        }
+    }};
+}
+
+#[derive(Debug, Clone, Copy)]
+enum Statistic {
+    Min,
+    Max,
+}
+
+/// Extracts statistics for a single leaf column from [`RowGroupMetaData`] as an
+/// arrow [`ArrayRef`]
+///
+/// For example, given a parquet file with 3 Row Groups, when asked for
+/// statistics for column "A" it will return a single array with 3 elements.
+pub(crate) struct RowGroupStatisticsConverter<'a> {
+    field: &'a Field,
+}
+
+impl<'a> RowGroupStatisticsConverter<'a> {
+    /// Create a new RowGoupStatisticsConverter for extracting the named column
+    /// in the schema.
+    pub fn try_new(schema: &'a arrow_schema::Schema, column_name: &str) -> Result<Self> {
+        let field = schema.field_with_name(column_name)?;
+        Ok(Self { field })
+    }
+
+    /// Returns the min value for the column into an array ref.
+    pub fn min<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(Statistic::Min, row_group_meta_data)
+    }
+
+    /// Returns the max value for the column into an array ref.
+    pub fn max<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(Statistic::Max, row_group_meta_data)
+    }
+
+    /// Extracts all min/max values for the column into an array ref.
+    fn min_max_impl<'b>(
+        &self,
+        mm: Statistic,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        let mut row_group_meta_data = row_group_meta_data.into_iter().peekable();
+
+        // if it is empty, return empty array
+        if row_group_meta_data.peek().is_none() {
+            return Ok(new_empty_array(self.field.data_type()));
+        }
+
+        let maybe_index = row_group_meta_data.peek().and_then(|rg_meta| {
+            rg_meta
+                .columns()
+                .iter()
+                .enumerate()
+                .find(|(_idx, c)| c.column_descr().name() == self.field.name())
+                .map(|(idx, _c)| idx)
+        });
+
+        // don't have this column, return an array of all NULLs
+        let Some(column_index) = maybe_index else {
+            let num_row_groups = row_group_meta_data.count();
+            let sv = ScalarValue::try_from(self.field.data_type())?;
+            return sv.to_array_of_size(num_row_groups);
+        };
+
+        let stats_iter = row_group_meta_data.map(move |row_group_meta_data| {
+            row_group_meta_data.column(column_index).statistics()
+        });
+
+        // this is the value to use when the statistics are not set
+        let null_value = ScalarValue::try_from(self.field.data_type())?;
+        match mm {
+            Statistic::Min => {
+                let values = stats_iter.map(|column_statistics| {
+                    column_statistics
+                        .and_then(|column_statistics| {
+                            get_statistic!(
+                                column_statistics,
+                                min,
+                                min_bytes,
+                                Some(self.field.data_type().clone())
+                            )
+                        })
+                        .unwrap_or_else(|| null_value.clone())
+                });
+                ScalarValue::iter_to_array(values)
+            }
+            Statistic::Max => {
+                let values = stats_iter.map(|column_statistics| {
+                    column_statistics
+                        .and_then(|column_statistics| {
+                            get_statistic!(
+                                column_statistics,
+                                max,
+                                max_bytes,
+                                Some(self.field.data_type().clone())
+                            )
+                        })
+                        .unwrap_or_else(|| null_value.clone())
+                });
+                ScalarValue::iter_to_array(values)
+            }
+        }
+    }
+}
+
+#[cfg(test)]
+mod test {
+    use super::*;
+    use arrow_array::{
+        new_null_array, Array, BinaryArray, BooleanArray, Decimal128Array, Float32Array,
+        Float64Array, Int32Array, Int64Array, RecordBatch, StringArray, StructArray,
+        TimestampNanosecondArray,
+    };
+    use arrow_schema::SchemaRef;
+    use bytes::Bytes;
+    use datafusion_common::test_util::parquet_test_data;
+    use parquet::arrow::arrow_reader::ArrowReaderBuilder;
+    use parquet::arrow::arrow_writer::ArrowWriter;
+    use parquet::file::metadata::ParquetMetaData;
+    use parquet::file::properties::{EnabledStatistics, WriterProperties};
+    use std::path::PathBuf;
+    use std::sync::Arc;
+
+    // TODO error cases (with parquet statistics that are mismatched in expected type)
+
+    #[test]
+    fn roundtrip_empty() {
+        let empty_bool_array = new_empty_array(&DataType::Boolean);
+        Test {
+            input: empty_bool_array.clone(),
+            expected_min: empty_bool_array.clone(),
+            expected_max: empty_bool_array.clone(),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_bool() {
+        Test {
+            input: bool_array([
+                // row group 1
+                Some(true),
+                None,
+                Some(true),
+                // row group 2
+                Some(true),
+                Some(false),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ]),
+            expected_min: bool_array([Some(true), Some(false), None]),
+            expected_max: bool_array([Some(true), Some(true), None]),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_int32() {
+        Test {
+            input: i32_array([
+                // row group 1
+                Some(1),
+                None,
+                Some(3),
+                // row group 2
+                Some(0),
+                Some(5),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ]),
+            expected_min: i32_array([Some(1), Some(0), None]),
+            expected_max: i32_array([Some(3), Some(5), None]),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_int64() {
+        Test {
+            input: i64_array([
+                // row group 1
+                Some(1),
+                None,
+                Some(3),
+                // row group 2
+                Some(0),
+                Some(5),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ]),
+            expected_min: i64_array([Some(1), Some(0), None]),
+            expected_max: i64_array(vec![Some(3), Some(5), None]),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_f32() {
+        Test {
+            input: f32_array([
+                // row group 1
+                Some(1.0),
+                None,
+                Some(3.0),
+                // row group 2
+                Some(-1.0),
+                Some(5.0),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ]),
+            expected_min: f32_array([Some(1.0), Some(-1.0), None]),
+            expected_max: f32_array([Some(3.0), Some(5.0), None]),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_f64() {
+        Test {
+            input: f64_array([
+                // row group 1
+                Some(1.0),
+                None,
+                Some(3.0),
+                // row group 2
+                Some(-1.0),
+                Some(5.0),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ]),
+            expected_min: f64_array([Some(1.0), Some(-1.0), None]),
+            expected_max: f64_array([Some(3.0), Some(5.0), None]),
+        }
+        .run()
+    }
+
+    #[test]
+    #[should_panic(
+        expected = "Inconsistent types in ScalarValue::iter_to_array. Expected Int64, got TimestampNanosecond(NULL, None)"
+    )]
+    // Due to https://github.com/apache/arrow-datafusion/issues/8295
+    fn roundtrip_timestamp() {
+        Test {
+            input: timestamp_array([
+                // row group 1
+                Some(1),
+                None,
+                Some(3),
+                // row group 2
+                Some(9),
+                Some(5),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ]),
+            expected_min: timestamp_array([Some(1), Some(5), None]),
+            expected_max: timestamp_array([Some(3), Some(9), None]),
+        }
+        .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]
+    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 {

Review Comment:
   added https://github.com/apache/arrow-datafusion/pull/8294/commits/a601fbffcf3e884f84f80f2058b743483c1e1937 which shows the error



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Extract parquet statistics to its own module, add tests [arrow-datafusion]

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on code in PR #8294:
URL: https://github.com/apache/arrow-datafusion/pull/8294#discussion_r1406731439


##########
datafusion/core/src/datasource/physical_plan/parquet/statistics.rs:
##########
@@ -0,0 +1,807 @@
+// 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.
+
+//! [`RowGroupStatisticsConverter`] tp converts parquet RowGroup statistics to arrow [`ArrayRef`].
+
+use arrow::{array::ArrayRef, datatypes::DataType};
+use arrow_array::new_empty_array;
+use arrow_schema::Field;
+use datafusion_common::{Result, ScalarValue};
+use parquet::file::{
+    metadata::RowGroupMetaData, statistics::Statistics as ParquetStatistics,
+};
+
+// 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(b))
+}
+
+// Copy from arrow-rs
+// https://github.com/apache/arrow-rs/blob/733b7e7fd1e8c43a404c3ce40ecf741d493c21b4/parquet/src/arrow/buffer/bit_util.rs#L55
+// Convert the byte slice to fixed length byte array with the length of 16
+fn sign_extend_be(b: &[u8]) -> [u8; 16] {
+    assert!(b.len() <= 16, "Array too large, expected less than 16");
+    let is_negative = (b[0] & 128u8) == 128u8;
+    let mut result = if is_negative { [255u8; 16] } else { [0u8; 16] };
+    for (d, s) in result.iter_mut().skip(16 - b.len()).zip(b) {
+        *d = *s;
+    }
+    result
+}
+
+/// Converts parquet RowGroup statistics (stored in
+/// [`RowGroupMetaData`]) into an arrow [`ArrayRef`]
+///
+/// For example, given a parquet file with 3 Row Groups, when asked for
+/// statistics for column "A" it will return a single array with 3 elements,
+///
+pub(crate) struct RowGroupStatisticsConverter<'a> {
+    field: &'a Field,
+}
+
+/// Extract a single min/max statistics from a [`ParquetStatistics`] object
+///
+/// * `$column_statistics` is the `ParquetStatistics` object
+/// * `$func is the function` (`min`/`max`) to call to get the value
+/// * `$bytes_func` is the function (`min_bytes`/`max_bytes`) to call to get the value as bytes
+/// * `$target_arrow_type` is the [`DataType`] of the target statistics
+macro_rules! get_statistic {
+    ($column_statistics:expr, $func:ident, $bytes_func:ident, $target_arrow_type:expr) => {{
+        if !$column_statistics.has_min_max_set() {
+            return None;
+        }
+        match $column_statistics {
+            ParquetStatistics::Boolean(s) => Some(ScalarValue::Boolean(Some(*s.$func()))),
+            ParquetStatistics::Int32(s) => {
+                match $target_arrow_type {
+                    // int32 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int32(Some(*s.$func()))),
+                }
+            }
+            ParquetStatistics::Int64(s) => {
+                match $target_arrow_type {
+                    // int64 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int64(Some(*s.$func()))),
+                }
+            }
+            // 96 bit ints not supported
+            ParquetStatistics::Int96(_) => None,
+            ParquetStatistics::Float(s) => Some(ScalarValue::Float32(Some(*s.$func()))),
+            ParquetStatistics::Double(s) => Some(ScalarValue::Float64(Some(*s.$func()))),
+            ParquetStatistics::ByteArray(s) => {
+                match $target_arrow_type {
+                    // decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => {
+                        let s = std::str::from_utf8(s.$bytes_func())
+                            .map(|s| s.to_string())
+                            .ok();
+                        Some(ScalarValue::Utf8(s))
+                    }
+                }
+            }
+            // type not supported yet
+            ParquetStatistics::FixedLenByteArray(s) => {
+                match $target_arrow_type {
+                    // just support the decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => None,
+                }
+            }
+        }
+    }};
+}
+
+#[derive(Debug, Clone, Copy)]
+enum MinMax {
+    Min,
+    Max,
+}
+
+impl<'a> RowGroupStatisticsConverter<'a> {
+    /// Create a new RowGoupStatisticsConverter suitable that can extract
+    /// statistics for the specified field
+    pub fn new(field: &'a Field) -> Self {
+        Self { field }
+    }
+
+    /// Returns the min value for the column into an array ref.
+    pub fn min<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(MinMax::Min, row_group_meta_data)
+    }
+
+    /// Returns the max value for the column into an array ref.
+    pub fn max<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(MinMax::Max, row_group_meta_data)
+    }
+
+    /// Extracts all min/max values for the column into an array ref.
+    fn min_max_impl<'b>(
+        &self,
+        mm: MinMax,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        let mut row_group_meta_data = row_group_meta_data.into_iter().peekable();
+
+        // if it is empty, return empty array
+        if row_group_meta_data.peek().is_none() {
+            return Ok(new_empty_array(self.field.data_type()));
+        }
+
+        let maybe_index = row_group_meta_data.peek().and_then(|rg_meta| {
+            rg_meta
+                .columns()
+                .iter()
+                .enumerate()
+                .find(|(_idx, c)| c.column_descr().name() == self.field.name())

Review Comment:
   I added tests for this -- and I didn't find a bug 🤔 



##########
datafusion/core/src/datasource/physical_plan/parquet/statistics.rs:
##########
@@ -0,0 +1,807 @@
+// 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.
+
+//! [`RowGroupStatisticsConverter`] tp converts parquet RowGroup statistics to arrow [`ArrayRef`].
+
+use arrow::{array::ArrayRef, datatypes::DataType};
+use arrow_array::new_empty_array;
+use arrow_schema::Field;
+use datafusion_common::{Result, ScalarValue};
+use parquet::file::{
+    metadata::RowGroupMetaData, statistics::Statistics as ParquetStatistics,
+};
+
+// 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(b))
+}
+
+// Copy from arrow-rs
+// https://github.com/apache/arrow-rs/blob/733b7e7fd1e8c43a404c3ce40ecf741d493c21b4/parquet/src/arrow/buffer/bit_util.rs#L55
+// Convert the byte slice to fixed length byte array with the length of 16
+fn sign_extend_be(b: &[u8]) -> [u8; 16] {
+    assert!(b.len() <= 16, "Array too large, expected less than 16");
+    let is_negative = (b[0] & 128u8) == 128u8;
+    let mut result = if is_negative { [255u8; 16] } else { [0u8; 16] };
+    for (d, s) in result.iter_mut().skip(16 - b.len()).zip(b) {
+        *d = *s;
+    }
+    result
+}
+
+/// Converts parquet RowGroup statistics (stored in
+/// [`RowGroupMetaData`]) into an arrow [`ArrayRef`]
+///
+/// For example, given a parquet file with 3 Row Groups, when asked for
+/// statistics for column "A" it will return a single array with 3 elements,
+///
+pub(crate) struct RowGroupStatisticsConverter<'a> {
+    field: &'a Field,
+}
+
+/// Extract a single min/max statistics from a [`ParquetStatistics`] object
+///
+/// * `$column_statistics` is the `ParquetStatistics` object
+/// * `$func is the function` (`min`/`max`) to call to get the value
+/// * `$bytes_func` is the function (`min_bytes`/`max_bytes`) to call to get the value as bytes
+/// * `$target_arrow_type` is the [`DataType`] of the target statistics
+macro_rules! get_statistic {
+    ($column_statistics:expr, $func:ident, $bytes_func:ident, $target_arrow_type:expr) => {{
+        if !$column_statistics.has_min_max_set() {
+            return None;
+        }
+        match $column_statistics {
+            ParquetStatistics::Boolean(s) => Some(ScalarValue::Boolean(Some(*s.$func()))),
+            ParquetStatistics::Int32(s) => {
+                match $target_arrow_type {
+                    // int32 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int32(Some(*s.$func()))),
+                }
+            }
+            ParquetStatistics::Int64(s) => {
+                match $target_arrow_type {
+                    // int64 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int64(Some(*s.$func()))),
+                }
+            }
+            // 96 bit ints not supported
+            ParquetStatistics::Int96(_) => None,
+            ParquetStatistics::Float(s) => Some(ScalarValue::Float32(Some(*s.$func()))),
+            ParquetStatistics::Double(s) => Some(ScalarValue::Float64(Some(*s.$func()))),
+            ParquetStatistics::ByteArray(s) => {
+                match $target_arrow_type {
+                    // decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => {
+                        let s = std::str::from_utf8(s.$bytes_func())
+                            .map(|s| s.to_string())
+                            .ok();
+                        Some(ScalarValue::Utf8(s))
+                    }
+                }
+            }
+            // type not supported yet
+            ParquetStatistics::FixedLenByteArray(s) => {
+                match $target_arrow_type {
+                    // just support the decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => None,
+                }
+            }
+        }
+    }};
+}
+
+#[derive(Debug, Clone, Copy)]
+enum MinMax {
+    Min,
+    Max,
+}
+
+impl<'a> RowGroupStatisticsConverter<'a> {
+    /// Create a new RowGoupStatisticsConverter suitable that can extract
+    /// statistics for the specified field
+    pub fn new(field: &'a Field) -> Self {
+        Self { field }
+    }
+
+    /// Returns the min value for the column into an array ref.
+    pub fn min<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(MinMax::Min, row_group_meta_data)
+    }
+
+    /// Returns the max value for the column into an array ref.
+    pub fn max<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(MinMax::Max, row_group_meta_data)
+    }
+
+    /// Extracts all min/max values for the column into an array ref.
+    fn min_max_impl<'b>(
+        &self,
+        mm: MinMax,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        let mut row_group_meta_data = row_group_meta_data.into_iter().peekable();
+
+        // if it is empty, return empty array
+        if row_group_meta_data.peek().is_none() {
+            return Ok(new_empty_array(self.field.data_type()));
+        }
+
+        let maybe_index = row_group_meta_data.peek().and_then(|rg_meta| {
+            rg_meta
+                .columns()
+                .iter()
+                .enumerate()
+                .find(|(_idx, c)| c.column_descr().name() == self.field.name())
+                .map(|(idx, _c)| idx)
+        });
+
+        // don't have this column, return an array of all NULLs
+        let Some(column_index) = maybe_index else {
+            let num_row_groups = row_group_meta_data.count();
+            let sv = ScalarValue::try_from(self.field.data_type())?;
+            return sv.to_array_of_size(num_row_groups);
+        };
+
+        let stats_iter = row_group_meta_data.map(move |row_group_meta_data| {
+            row_group_meta_data.column(column_index).statistics()
+        });
+
+        // this is the value to use when the statistics are not set
+        let null_value = ScalarValue::try_from(self.field.data_type())?;
+        match mm {
+            MinMax::Min => {
+                let values = stats_iter.map(|column_statistics| {
+                    column_statistics
+                        .and_then(|column_statistics| {
+                            get_statistic!(
+                                column_statistics,
+                                min,
+                                min_bytes,
+                                Some(self.field.data_type().clone())
+                            )
+                        })
+                        .unwrap_or_else(|| null_value.clone())
+                });
+                ScalarValue::iter_to_array(values)
+            }
+            MinMax::Max => {
+                let values = stats_iter.map(|column_statistics| {
+                    column_statistics
+                        .and_then(|column_statistics| {
+                            get_statistic!(
+                                column_statistics,
+                                max,
+                                max_bytes,
+                                Some(self.field.data_type().clone())
+                            )
+                        })
+                        .unwrap_or_else(|| null_value.clone())
+                });
+                ScalarValue::iter_to_array(values)
+            }
+        }
+    }
+}
+
+#[cfg(test)]
+mod test {
+    use super::*;
+    use arrow_array::{
+        BinaryArray, BooleanArray, Decimal128Array, Float32Array, Float64Array,
+        Int32Array, Int64Array, RecordBatch, StringArray, TimestampNanosecondArray,
+    };
+    use arrow_schema::SchemaRef;
+    use bytes::Bytes;
+    use datafusion_common::test_util::parquet_test_data;
+    use parquet::arrow::arrow_reader::ArrowReaderBuilder;
+    use parquet::arrow::arrow_writer::ArrowWriter;
+    use parquet::file::metadata::ParquetMetaData;
+    use parquet::file::properties::{EnabledStatistics, WriterProperties};
+    use std::path::PathBuf;
+    use std::sync::Arc;
+
+    // TODO error cases (with parquet statistics that are mismatched in expected type)
+

Review Comment:
   I added a test showing how nested structs are handled:
   
   * `fn roundtrip_struct() {`: Structs appear to be read out as all NULL, which is ok. 
   * `fn struct_and_non_struct() {`: Ensures that statistics for an array that appears *after* a struct array are not wrong



##########
datafusion/core/src/datasource/physical_plan/parquet/statistics.rs:
##########
@@ -0,0 +1,807 @@
+// 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.
+
+//! [`RowGroupStatisticsConverter`] tp converts parquet RowGroup statistics to arrow [`ArrayRef`].
+
+use arrow::{array::ArrayRef, datatypes::DataType};
+use arrow_array::new_empty_array;
+use arrow_schema::Field;
+use datafusion_common::{Result, ScalarValue};
+use parquet::file::{
+    metadata::RowGroupMetaData, statistics::Statistics as ParquetStatistics,
+};
+
+// 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(b))
+}
+
+// Copy from arrow-rs
+// https://github.com/apache/arrow-rs/blob/733b7e7fd1e8c43a404c3ce40ecf741d493c21b4/parquet/src/arrow/buffer/bit_util.rs#L55
+// Convert the byte slice to fixed length byte array with the length of 16
+fn sign_extend_be(b: &[u8]) -> [u8; 16] {
+    assert!(b.len() <= 16, "Array too large, expected less than 16");
+    let is_negative = (b[0] & 128u8) == 128u8;
+    let mut result = if is_negative { [255u8; 16] } else { [0u8; 16] };
+    for (d, s) in result.iter_mut().skip(16 - b.len()).zip(b) {
+        *d = *s;
+    }
+    result
+}
+
+/// Converts parquet RowGroup statistics (stored in
+/// [`RowGroupMetaData`]) into an arrow [`ArrayRef`]
+///
+/// For example, given a parquet file with 3 Row Groups, when asked for
+/// statistics for column "A" it will return a single array with 3 elements,
+///
+pub(crate) struct RowGroupStatisticsConverter<'a> {
+    field: &'a Field,
+}
+
+/// Extract a single min/max statistics from a [`ParquetStatistics`] object
+///
+/// * `$column_statistics` is the `ParquetStatistics` object
+/// * `$func is the function` (`min`/`max`) to call to get the value
+/// * `$bytes_func` is the function (`min_bytes`/`max_bytes`) to call to get the value as bytes
+/// * `$target_arrow_type` is the [`DataType`] of the target statistics
+macro_rules! get_statistic {
+    ($column_statistics:expr, $func:ident, $bytes_func:ident, $target_arrow_type:expr) => {{
+        if !$column_statistics.has_min_max_set() {
+            return None;
+        }
+        match $column_statistics {
+            ParquetStatistics::Boolean(s) => Some(ScalarValue::Boolean(Some(*s.$func()))),
+            ParquetStatistics::Int32(s) => {
+                match $target_arrow_type {
+                    // int32 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int32(Some(*s.$func()))),
+                }
+            }
+            ParquetStatistics::Int64(s) => {
+                match $target_arrow_type {
+                    // int64 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int64(Some(*s.$func()))),
+                }
+            }
+            // 96 bit ints not supported
+            ParquetStatistics::Int96(_) => None,
+            ParquetStatistics::Float(s) => Some(ScalarValue::Float32(Some(*s.$func()))),
+            ParquetStatistics::Double(s) => Some(ScalarValue::Float64(Some(*s.$func()))),
+            ParquetStatistics::ByteArray(s) => {
+                match $target_arrow_type {
+                    // decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => {
+                        let s = std::str::from_utf8(s.$bytes_func())
+                            .map(|s| s.to_string())
+                            .ok();
+                        Some(ScalarValue::Utf8(s))
+                    }
+                }
+            }
+            // type not supported yet
+            ParquetStatistics::FixedLenByteArray(s) => {
+                match $target_arrow_type {
+                    // just support the decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => None,
+                }
+            }
+        }
+    }};
+}
+
+#[derive(Debug, Clone, Copy)]
+enum MinMax {
+    Min,
+    Max,
+}
+
+impl<'a> RowGroupStatisticsConverter<'a> {

Review Comment:
   I have updated this API to be in terms of a field and encapsulated the "find a named arrow field in in the parquet schema" into the constructor. 
   
   I think this is the natural API for users who are reading the table out as RecordBatches and so are referring to columns by their name (in the arrow `Schema`)



##########
datafusion/core/src/datasource/physical_plan/parquet/statistics.rs:
##########
@@ -0,0 +1,807 @@
+// 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.
+
+//! [`RowGroupStatisticsConverter`] tp converts parquet RowGroup statistics to arrow [`ArrayRef`].
+
+use arrow::{array::ArrayRef, datatypes::DataType};
+use arrow_array::new_empty_array;
+use arrow_schema::Field;
+use datafusion_common::{Result, ScalarValue};
+use parquet::file::{
+    metadata::RowGroupMetaData, statistics::Statistics as ParquetStatistics,
+};
+
+// 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(b))
+}
+
+// Copy from arrow-rs
+// https://github.com/apache/arrow-rs/blob/733b7e7fd1e8c43a404c3ce40ecf741d493c21b4/parquet/src/arrow/buffer/bit_util.rs#L55
+// Convert the byte slice to fixed length byte array with the length of 16
+fn sign_extend_be(b: &[u8]) -> [u8; 16] {
+    assert!(b.len() <= 16, "Array too large, expected less than 16");
+    let is_negative = (b[0] & 128u8) == 128u8;
+    let mut result = if is_negative { [255u8; 16] } else { [0u8; 16] };
+    for (d, s) in result.iter_mut().skip(16 - b.len()).zip(b) {
+        *d = *s;
+    }
+    result
+}
+
+/// Converts parquet RowGroup statistics (stored in
+/// [`RowGroupMetaData`]) into an arrow [`ArrayRef`]
+///
+/// For example, given a parquet file with 3 Row Groups, when asked for
+/// statistics for column "A" it will return a single array with 3 elements,
+///
+pub(crate) struct RowGroupStatisticsConverter<'a> {
+    field: &'a Field,
+}
+
+/// Extract a single min/max statistics from a [`ParquetStatistics`] object
+///
+/// * `$column_statistics` is the `ParquetStatistics` object
+/// * `$func is the function` (`min`/`max`) to call to get the value
+/// * `$bytes_func` is the function (`min_bytes`/`max_bytes`) to call to get the value as bytes
+/// * `$target_arrow_type` is the [`DataType`] of the target statistics
+macro_rules! get_statistic {
+    ($column_statistics:expr, $func:ident, $bytes_func:ident, $target_arrow_type:expr) => {{
+        if !$column_statistics.has_min_max_set() {
+            return None;
+        }
+        match $column_statistics {
+            ParquetStatistics::Boolean(s) => Some(ScalarValue::Boolean(Some(*s.$func()))),
+            ParquetStatistics::Int32(s) => {
+                match $target_arrow_type {
+                    // int32 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int32(Some(*s.$func()))),
+                }
+            }
+            ParquetStatistics::Int64(s) => {
+                match $target_arrow_type {
+                    // int64 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int64(Some(*s.$func()))),
+                }
+            }
+            // 96 bit ints not supported
+            ParquetStatistics::Int96(_) => None,
+            ParquetStatistics::Float(s) => Some(ScalarValue::Float32(Some(*s.$func()))),
+            ParquetStatistics::Double(s) => Some(ScalarValue::Float64(Some(*s.$func()))),
+            ParquetStatistics::ByteArray(s) => {
+                match $target_arrow_type {
+                    // decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => {
+                        let s = std::str::from_utf8(s.$bytes_func())
+                            .map(|s| s.to_string())
+                            .ok();
+                        Some(ScalarValue::Utf8(s))
+                    }
+                }
+            }
+            // type not supported yet
+            ParquetStatistics::FixedLenByteArray(s) => {
+                match $target_arrow_type {
+                    // just support the decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => None,
+                }
+            }
+        }
+    }};
+}
+
+#[derive(Debug, Clone, Copy)]
+enum MinMax {
+    Min,
+    Max,
+}
+
+impl<'a> RowGroupStatisticsConverter<'a> {
+    /// Create a new RowGoupStatisticsConverter suitable that can extract
+    /// statistics for the specified field
+    pub fn new(field: &'a Field) -> Self {
+        Self { field }
+    }
+
+    /// Returns the min value for the column into an array ref.
+    pub fn min<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(MinMax::Min, row_group_meta_data)
+    }
+
+    /// Returns the max value for the column into an array ref.
+    pub fn max<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(MinMax::Max, row_group_meta_data)
+    }
+
+    /// Extracts all min/max values for the column into an array ref.
+    fn min_max_impl<'b>(
+        &self,
+        mm: MinMax,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        let mut row_group_meta_data = row_group_meta_data.into_iter().peekable();
+
+        // if it is empty, return empty array
+        if row_group_meta_data.peek().is_none() {
+            return Ok(new_empty_array(self.field.data_type()));
+        }
+
+        let maybe_index = row_group_meta_data.peek().and_then(|rg_meta| {
+            rg_meta
+                .columns()
+                .iter()
+                .enumerate()
+                .find(|(_idx, c)| c.column_descr().name() == self.field.name())

Review Comment:
   I added tests for this -- and I didn't find a bug 🤔 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Extract parquet statistics to its own module, add tests [arrow-datafusion]

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on code in PR #8294:
URL: https://github.com/apache/arrow-datafusion/pull/8294#discussion_r1406585965


##########
datafusion/core/src/datasource/physical_plan/parquet/statistics.rs:
##########
@@ -0,0 +1,807 @@
+// 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.
+
+//! [`RowGroupStatisticsConverter`] tp converts parquet RowGroup statistics to arrow [`ArrayRef`].
+
+use arrow::{array::ArrayRef, datatypes::DataType};
+use arrow_array::new_empty_array;
+use arrow_schema::Field;
+use datafusion_common::{Result, ScalarValue};
+use parquet::file::{
+    metadata::RowGroupMetaData, statistics::Statistics as ParquetStatistics,
+};
+
+// 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(b))
+}
+
+// Copy from arrow-rs
+// https://github.com/apache/arrow-rs/blob/733b7e7fd1e8c43a404c3ce40ecf741d493c21b4/parquet/src/arrow/buffer/bit_util.rs#L55
+// Convert the byte slice to fixed length byte array with the length of 16
+fn sign_extend_be(b: &[u8]) -> [u8; 16] {
+    assert!(b.len() <= 16, "Array too large, expected less than 16");
+    let is_negative = (b[0] & 128u8) == 128u8;
+    let mut result = if is_negative { [255u8; 16] } else { [0u8; 16] };
+    for (d, s) in result.iter_mut().skip(16 - b.len()).zip(b) {
+        *d = *s;
+    }
+    result
+}
+
+/// Converts parquet RowGroup statistics (stored in
+/// [`RowGroupMetaData`]) into an arrow [`ArrayRef`]
+///
+/// For example, given a parquet file with 3 Row Groups, when asked for
+/// statistics for column "A" it will return a single array with 3 elements,
+///
+pub(crate) struct RowGroupStatisticsConverter<'a> {
+    field: &'a Field,
+}
+
+/// Extract a single min/max statistics from a [`ParquetStatistics`] object
+///
+/// * `$column_statistics` is the `ParquetStatistics` object
+/// * `$func is the function` (`min`/`max`) to call to get the value
+/// * `$bytes_func` is the function (`min_bytes`/`max_bytes`) to call to get the value as bytes
+/// * `$target_arrow_type` is the [`DataType`] of the target statistics
+macro_rules! get_statistic {
+    ($column_statistics:expr, $func:ident, $bytes_func:ident, $target_arrow_type:expr) => {{
+        if !$column_statistics.has_min_max_set() {
+            return None;
+        }
+        match $column_statistics {
+            ParquetStatistics::Boolean(s) => Some(ScalarValue::Boolean(Some(*s.$func()))),
+            ParquetStatistics::Int32(s) => {
+                match $target_arrow_type {
+                    // int32 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int32(Some(*s.$func()))),
+                }
+            }
+            ParquetStatistics::Int64(s) => {
+                match $target_arrow_type {
+                    // int64 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int64(Some(*s.$func()))),
+                }
+            }
+            // 96 bit ints not supported
+            ParquetStatistics::Int96(_) => None,
+            ParquetStatistics::Float(s) => Some(ScalarValue::Float32(Some(*s.$func()))),
+            ParquetStatistics::Double(s) => Some(ScalarValue::Float64(Some(*s.$func()))),
+            ParquetStatistics::ByteArray(s) => {
+                match $target_arrow_type {
+                    // decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => {
+                        let s = std::str::from_utf8(s.$bytes_func())
+                            .map(|s| s.to_string())
+                            .ok();
+                        Some(ScalarValue::Utf8(s))
+                    }
+                }
+            }
+            // type not supported yet
+            ParquetStatistics::FixedLenByteArray(s) => {
+                match $target_arrow_type {
+                    // just support the decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => None,
+                }
+            }
+        }
+    }};
+}
+
+#[derive(Debug, Clone, Copy)]
+enum MinMax {
+    Min,
+    Max,
+}
+
+impl<'a> RowGroupStatisticsConverter<'a> {
+    /// Create a new RowGoupStatisticsConverter suitable that can extract
+    /// statistics for the specified field
+    pub fn new(field: &'a Field) -> Self {
+        Self { field }
+    }
+
+    /// Returns the min value for the column into an array ref.
+    pub fn min<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(MinMax::Min, row_group_meta_data)
+    }
+
+    /// Returns the max value for the column into an array ref.
+    pub fn max<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(MinMax::Max, row_group_meta_data)
+    }
+
+    /// Extracts all min/max values for the column into an array ref.
+    fn min_max_impl<'b>(
+        &self,
+        mm: MinMax,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        let mut row_group_meta_data = row_group_meta_data.into_iter().peekable();
+
+        // if it is empty, return empty array
+        if row_group_meta_data.peek().is_none() {
+            return Ok(new_empty_array(self.field.data_type()));
+        }
+
+        let maybe_index = row_group_meta_data.peek().and_then(|rg_meta| {
+            rg_meta
+                .columns()
+                .iter()
+                .enumerate()
+                .find(|(_idx, c)| c.column_descr().name() == self.field.name())
+                .map(|(idx, _c)| idx)
+        });
+
+        // don't have this column, return an array of all NULLs
+        let Some(column_index) = maybe_index else {
+            let num_row_groups = row_group_meta_data.count();
+            let sv = ScalarValue::try_from(self.field.data_type())?;
+            return sv.to_array_of_size(num_row_groups);
+        };
+
+        let stats_iter = row_group_meta_data.map(move |row_group_meta_data| {
+            row_group_meta_data.column(column_index).statistics()
+        });
+
+        // this is the value to use when the statistics are not set
+        let null_value = ScalarValue::try_from(self.field.data_type())?;
+        match mm {
+            MinMax::Min => {
+                let values = stats_iter.map(|column_statistics| {
+                    column_statistics
+                        .and_then(|column_statistics| {
+                            get_statistic!(
+                                column_statistics,
+                                min,
+                                min_bytes,
+                                Some(self.field.data_type().clone())
+                            )
+                        })
+                        .unwrap_or_else(|| null_value.clone())
+                });
+                ScalarValue::iter_to_array(values)
+            }
+            MinMax::Max => {
+                let values = stats_iter.map(|column_statistics| {
+                    column_statistics
+                        .and_then(|column_statistics| {
+                            get_statistic!(
+                                column_statistics,
+                                max,
+                                max_bytes,
+                                Some(self.field.data_type().clone())
+                            )
+                        })
+                        .unwrap_or_else(|| null_value.clone())
+                });
+                ScalarValue::iter_to_array(values)

Review Comment:
   Yes, exactly -- I plan to fix the code to build arrays directly, but I want to keep the current logic and add good test coverage before doing  so to avoid regressions



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Extract parquet statistics to its own module, add tests [arrow-datafusion]

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on code in PR #8294:
URL: https://github.com/apache/arrow-datafusion/pull/8294#discussion_r1406810742


##########
datafusion/core/src/datasource/physical_plan/parquet/statistics.rs:
##########
@@ -0,0 +1,925 @@
+// 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.
+
+//! [`RowGroupStatisticsConverter`] tp converts parquet RowGroup statistics to arrow [`ArrayRef`].
+
+use arrow::{array::ArrayRef, datatypes::DataType};
+use arrow_array::new_empty_array;
+use arrow_schema::Field;
+use datafusion_common::{Result, ScalarValue};
+use parquet::file::{
+    metadata::RowGroupMetaData, statistics::Statistics as ParquetStatistics,
+};
+
+// 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(b))
+}
+
+// Copy from arrow-rs
+// https://github.com/apache/arrow-rs/blob/733b7e7fd1e8c43a404c3ce40ecf741d493c21b4/parquet/src/arrow/buffer/bit_util.rs#L55
+// Convert the byte slice to fixed length byte array with the length of 16
+fn sign_extend_be(b: &[u8]) -> [u8; 16] {
+    assert!(b.len() <= 16, "Array too large, expected less than 16");
+    let is_negative = (b[0] & 128u8) == 128u8;
+    let mut result = if is_negative { [255u8; 16] } else { [0u8; 16] };
+    for (d, s) in result.iter_mut().skip(16 - b.len()).zip(b) {
+        *d = *s;
+    }
+    result
+}
+
+/// Extract a single min/max statistics from a [`ParquetStatistics`] object
+///
+/// * `$column_statistics` is the `ParquetStatistics` object
+/// * `$func is the function` (`min`/`max`) to call to get the value
+/// * `$bytes_func` is the function (`min_bytes`/`max_bytes`) to call to get the value as bytes
+/// * `$target_arrow_type` is the [`DataType`] of the target statistics
+macro_rules! get_statistic {
+    ($column_statistics:expr, $func:ident, $bytes_func:ident, $target_arrow_type:expr) => {{
+        if !$column_statistics.has_min_max_set() {
+            return None;
+        }
+        match $column_statistics {
+            ParquetStatistics::Boolean(s) => Some(ScalarValue::Boolean(Some(*s.$func()))),
+            ParquetStatistics::Int32(s) => {
+                match $target_arrow_type {
+                    // int32 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int32(Some(*s.$func()))),
+                }
+            }
+            ParquetStatistics::Int64(s) => {
+                match $target_arrow_type {
+                    // int64 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int64(Some(*s.$func()))),
+                }
+            }
+            // 96 bit ints not supported
+            ParquetStatistics::Int96(_) => None,
+            ParquetStatistics::Float(s) => Some(ScalarValue::Float32(Some(*s.$func()))),
+            ParquetStatistics::Double(s) => Some(ScalarValue::Float64(Some(*s.$func()))),
+            ParquetStatistics::ByteArray(s) => {
+                match $target_arrow_type {
+                    // decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => {
+                        let s = std::str::from_utf8(s.$bytes_func())
+                            .map(|s| s.to_string())
+                            .ok();
+                        Some(ScalarValue::Utf8(s))
+                    }
+                }
+            }
+            // type not supported yet
+            ParquetStatistics::FixedLenByteArray(s) => {
+                match $target_arrow_type {
+                    // just support the decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => None,
+                }
+            }
+        }
+    }};
+}
+
+#[derive(Debug, Clone, Copy)]
+enum Statistic {
+    Min,
+    Max,
+}
+
+/// Extracts statistics for a single leaf column from [`RowGroupMetaData`] as an
+/// arrow [`ArrayRef`]
+///
+/// For example, given a parquet file with 3 Row Groups, when asked for
+/// statistics for column "A" it will return a single array with 3 elements.
+pub(crate) struct RowGroupStatisticsConverter<'a> {
+    field: &'a Field,
+}
+
+impl<'a> RowGroupStatisticsConverter<'a> {
+    /// Create a new RowGoupStatisticsConverter for extracting the named column
+    /// in the schema.
+    pub fn try_new(schema: &'a arrow_schema::Schema, column_name: &str) -> Result<Self> {
+        let field = schema.field_with_name(column_name)?;
+        Ok(Self { field })
+    }
+
+    /// Returns the min value for the column into an array ref.
+    pub fn min<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(Statistic::Min, row_group_meta_data)
+    }
+
+    /// Returns the max value for the column into an array ref.
+    pub fn max<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(Statistic::Max, row_group_meta_data)
+    }
+
+    /// Extracts all min/max values for the column into an array ref.
+    fn min_max_impl<'b>(
+        &self,
+        mm: Statistic,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        let mut row_group_meta_data = row_group_meta_data.into_iter().peekable();
+
+        // if it is empty, return empty array
+        if row_group_meta_data.peek().is_none() {
+            return Ok(new_empty_array(self.field.data_type()));
+        }
+
+        let maybe_index = row_group_meta_data.peek().and_then(|rg_meta| {
+            rg_meta
+                .columns()
+                .iter()
+                .enumerate()
+                .find(|(_idx, c)| c.column_descr().name() == self.field.name())
+                .map(|(idx, _c)| idx)
+        });
+
+        // don't have this column, return an array of all NULLs
+        let Some(column_index) = maybe_index else {
+            let num_row_groups = row_group_meta_data.count();
+            let sv = ScalarValue::try_from(self.field.data_type())?;
+            return sv.to_array_of_size(num_row_groups);
+        };
+
+        let stats_iter = row_group_meta_data.map(move |row_group_meta_data| {
+            row_group_meta_data.column(column_index).statistics()
+        });
+
+        // this is the value to use when the statistics are not set
+        let null_value = ScalarValue::try_from(self.field.data_type())?;
+        match mm {
+            Statistic::Min => {
+                let values = stats_iter.map(|column_statistics| {
+                    column_statistics
+                        .and_then(|column_statistics| {
+                            get_statistic!(
+                                column_statistics,
+                                min,
+                                min_bytes,
+                                Some(self.field.data_type().clone())
+                            )
+                        })
+                        .unwrap_or_else(|| null_value.clone())
+                });
+                ScalarValue::iter_to_array(values)
+            }
+            Statistic::Max => {
+                let values = stats_iter.map(|column_statistics| {
+                    column_statistics
+                        .and_then(|column_statistics| {
+                            get_statistic!(
+                                column_statistics,
+                                max,
+                                max_bytes,
+                                Some(self.field.data_type().clone())
+                            )
+                        })
+                        .unwrap_or_else(|| null_value.clone())
+                });
+                ScalarValue::iter_to_array(values)
+            }
+        }
+    }
+}
+
+#[cfg(test)]
+mod test {
+    use super::*;
+    use arrow_array::{
+        new_null_array, Array, BinaryArray, BooleanArray, Decimal128Array, Float32Array,
+        Float64Array, Int32Array, Int64Array, RecordBatch, StringArray, StructArray,
+        TimestampNanosecondArray,
+    };
+    use arrow_schema::SchemaRef;
+    use bytes::Bytes;
+    use datafusion_common::test_util::parquet_test_data;
+    use parquet::arrow::arrow_reader::ArrowReaderBuilder;
+    use parquet::arrow::arrow_writer::ArrowWriter;
+    use parquet::file::metadata::ParquetMetaData;
+    use parquet::file::properties::{EnabledStatistics, WriterProperties};
+    use std::path::PathBuf;
+    use std::sync::Arc;
+
+    // TODO error cases (with parquet statistics that are mismatched in expected type)
+
+    #[test]
+    fn roundtrip_empty() {
+        let empty_bool_array = new_empty_array(&DataType::Boolean);
+        Test {
+            input: empty_bool_array.clone(),
+            expected_min: empty_bool_array.clone(),
+            expected_max: empty_bool_array.clone(),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_bool() {
+        Test {
+            input: bool_array([
+                // row group 1
+                Some(true),
+                None,
+                Some(true),
+                // row group 2
+                Some(true),
+                Some(false),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ]),
+            expected_min: bool_array([Some(true), Some(false), None]),
+            expected_max: bool_array([Some(true), Some(true), None]),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_int32() {
+        Test {
+            input: i32_array([
+                // row group 1
+                Some(1),
+                None,
+                Some(3),
+                // row group 2
+                Some(0),
+                Some(5),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ]),
+            expected_min: i32_array([Some(1), Some(0), None]),
+            expected_max: i32_array([Some(3), Some(5), None]),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_int64() {
+        Test {
+            input: i64_array([
+                // row group 1
+                Some(1),
+                None,
+                Some(3),
+                // row group 2
+                Some(0),
+                Some(5),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ]),
+            expected_min: i64_array([Some(1), Some(0), None]),
+            expected_max: i64_array(vec![Some(3), Some(5), None]),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_f32() {
+        Test {
+            input: f32_array([
+                // row group 1
+                Some(1.0),
+                None,
+                Some(3.0),
+                // row group 2
+                Some(-1.0),
+                Some(5.0),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ]),
+            expected_min: f32_array([Some(1.0), Some(-1.0), None]),
+            expected_max: f32_array([Some(3.0), Some(5.0), None]),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_f64() {
+        Test {
+            input: f64_array([
+                // row group 1
+                Some(1.0),
+                None,
+                Some(3.0),
+                // row group 2
+                Some(-1.0),
+                Some(5.0),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ]),
+            expected_min: f64_array([Some(1.0), Some(-1.0), None]),
+            expected_max: f64_array([Some(3.0), Some(5.0), None]),
+        }
+        .run()
+    }
+
+    #[test]
+    #[should_panic(
+        expected = "Inconsistent types in ScalarValue::iter_to_array. Expected Int64, got TimestampNanosecond(NULL, None)"
+    )]
+    // Due to https://github.com/apache/arrow-datafusion/issues/8295
+    fn roundtrip_timestamp() {
+        Test {
+            input: timestamp_array([
+                // row group 1
+                Some(1),
+                None,
+                Some(3),
+                // row group 2
+                Some(9),
+                Some(5),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ]),
+            expected_min: timestamp_array([Some(1), Some(5), None]),
+            expected_max: timestamp_array([Some(3), Some(9), None]),
+        }
+        .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]
+    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/arrow-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]
+    #[should_panic(
+        expected = "Inconsistent types in ScalarValue::iter_to_array. Expected Utf8, got Binary(NULL)"
+    )]
+    // Due to https://github.com/apache/arrow-datafusion/issues/8295
+    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 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)]);
+
+        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);
+
+        // read the int_col statistics
+        let (_idx, field) = schema.column_with_name("int_col").unwrap();
+
+        let converter =
+            RowGroupStatisticsConverter::try_new(&schema, field.name()).unwrap();
+        let row_groups = metadata.row_groups();
+        let min = converter.min(row_groups).unwrap();
+        assert_eq!(
+            &min,
+            &expected_min,
+            "Min. Statistics\n\n{}\n\n",
+            DisplayStats(row_groups)
+        );
+
+        let max = converter.max(row_groups).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: i32_array([Some(0)]),
+                expected_max: i32_array([Some(9)]),
+            })
+            .with_column(ExpectedColumn {
+                name: "smallint_col",
+                expected_min: i32_array([Some(0)]),
+                expected_max: i32_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_array([None]),
+                expected_max: timestamp_array([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);
+
+            for field in schema.fields() {
+                let converter =
+                    RowGroupStatisticsConverter::try_new(&schema, field.name()).unwrap();
+                let row_groups = metadata.row_groups();
+                let min = converter.min(row_groups).unwrap();
+                assert_eq!(
+                    &min,
+                    &expected_min,
+                    "Min. Statistics\n\n{}\n\n",
+                    DisplayStats(row_groups)
+                );
+
+                let max = converter.max(row_groups).unwrap();
+                assert_eq!(
+                    &max,
+                    &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<ParquetMetaData> {
+        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<ExpectedColumn>,
+    }
+
+    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 exepcted 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();
+
+            for expected_column in self.expected_columns {
+                let ExpectedColumn {
+                    name,
+                    expected_min,
+                    expected_max,
+                } = expected_column;
+
+                let converter = RowGroupStatisticsConverter::try_new(arrow_schema, name)
+                    .expect("can't find field in schema");
+
+                let actual_min = converter.min(metadata.row_groups()).unwrap();
+                assert_eq!(&expected_min, &actual_min, "column {name}");
+
+                let actual_max = converter.max(metadata.row_groups()).unwrap();
+                assert_eq!(&expected_max, &actual_max, "column {name}");
+            }
+        }
+    }
+
+    fn bool_array(input: impl IntoIterator<Item = Option<bool>>) -> ArrayRef {
+        let array: BooleanArray = input.into_iter().collect();
+        Arc::new(array)
+    }
+
+    fn i32_array(input: impl IntoIterator<Item = Option<i32>>) -> ArrayRef {
+        let array: Int32Array = input.into_iter().collect();
+        Arc::new(array)
+    }
+
+    fn i64_array(input: impl IntoIterator<Item = Option<i64>>) -> ArrayRef {
+        let array: Int64Array = input.into_iter().collect();
+        Arc::new(array)
+    }
+
+    fn f32_array(input: impl IntoIterator<Item = Option<f32>>) -> ArrayRef {
+        let array: Float32Array = input.into_iter().collect();
+        Arc::new(array)
+    }
+
+    fn f64_array(input: impl IntoIterator<Item = Option<f64>>) -> ArrayRef {
+        let array: Float64Array = input.into_iter().collect();
+        Arc::new(array)
+    }
+
+    fn timestamp_array(input: impl IntoIterator<Item = Option<i64>>) -> ArrayRef {
+        let array: TimestampNanosecondArray = input.into_iter().collect();
+        Arc::new(array)
+    }
+
+    fn utf8_array<'a>(input: impl IntoIterator<Item = Option<&'a str>>) -> 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 "b" and "i" with the specified values
+    fn struct_array(input: Vec<(Option<bool>, Option<i32>)>) -> 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("b", DataType::Boolean, nullable)),
+                Arc::new(boolean) as ArrayRef,
+            ),
+            (
+                Arc::new(Field::new("i", DataType::Int32, nullable)),

Review Comment:
   Indeed -- when I made this change in a601fbffc the test with structs and non structs fails (as you predicated)
   ```
   
     row_groups: 1
      "struct_col.bool_col": Boolean({min: Some(true), max: Some(true), distinct_count: None, null_count: 1, min_max_deprecated: false, min_max_backwards_compatible: false})
      "struct_col.int_col": Int32({min: Some(1), max: Some(3), distinct_count: None, null_count: 1, min_max_deprecated: false, min_max_backwards_compatible: false})
      "int_col": Int32({min: Some(100), max: Some(300), distinct_count: None, null_count: 0, min_max_deprecated: false, min_max_backwards_compatible: false})
   
   
   
     left: PrimitiveArray<Int32>
   [
     1,
   ]
    right: PrimitiveArray<Int32>
   [
     100,
   ]
   stack backtrace:
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Extract parquet statistics to its own module, add tests [arrow-datafusion]

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on code in PR #8294:
URL: https://github.com/apache/arrow-datafusion/pull/8294#discussion_r1406816141


##########
datafusion/core/src/datasource/physical_plan/parquet/statistics.rs:
##########
@@ -0,0 +1,925 @@
+// 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.
+
+//! [`RowGroupStatisticsConverter`] tp converts parquet RowGroup statistics to arrow [`ArrayRef`].
+
+use arrow::{array::ArrayRef, datatypes::DataType};
+use arrow_array::new_empty_array;
+use arrow_schema::Field;
+use datafusion_common::{Result, ScalarValue};
+use parquet::file::{
+    metadata::RowGroupMetaData, statistics::Statistics as ParquetStatistics,
+};
+
+// 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(b))
+}
+
+// Copy from arrow-rs
+// https://github.com/apache/arrow-rs/blob/733b7e7fd1e8c43a404c3ce40ecf741d493c21b4/parquet/src/arrow/buffer/bit_util.rs#L55
+// Convert the byte slice to fixed length byte array with the length of 16
+fn sign_extend_be(b: &[u8]) -> [u8; 16] {
+    assert!(b.len() <= 16, "Array too large, expected less than 16");
+    let is_negative = (b[0] & 128u8) == 128u8;
+    let mut result = if is_negative { [255u8; 16] } else { [0u8; 16] };
+    for (d, s) in result.iter_mut().skip(16 - b.len()).zip(b) {
+        *d = *s;
+    }
+    result
+}
+
+/// Extract a single min/max statistics from a [`ParquetStatistics`] object
+///
+/// * `$column_statistics` is the `ParquetStatistics` object
+/// * `$func is the function` (`min`/`max`) to call to get the value
+/// * `$bytes_func` is the function (`min_bytes`/`max_bytes`) to call to get the value as bytes
+/// * `$target_arrow_type` is the [`DataType`] of the target statistics
+macro_rules! get_statistic {
+    ($column_statistics:expr, $func:ident, $bytes_func:ident, $target_arrow_type:expr) => {{
+        if !$column_statistics.has_min_max_set() {
+            return None;
+        }
+        match $column_statistics {
+            ParquetStatistics::Boolean(s) => Some(ScalarValue::Boolean(Some(*s.$func()))),
+            ParquetStatistics::Int32(s) => {
+                match $target_arrow_type {
+                    // int32 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int32(Some(*s.$func()))),
+                }
+            }
+            ParquetStatistics::Int64(s) => {
+                match $target_arrow_type {
+                    // int64 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int64(Some(*s.$func()))),
+                }
+            }
+            // 96 bit ints not supported
+            ParquetStatistics::Int96(_) => None,
+            ParquetStatistics::Float(s) => Some(ScalarValue::Float32(Some(*s.$func()))),
+            ParquetStatistics::Double(s) => Some(ScalarValue::Float64(Some(*s.$func()))),
+            ParquetStatistics::ByteArray(s) => {
+                match $target_arrow_type {
+                    // decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => {
+                        let s = std::str::from_utf8(s.$bytes_func())
+                            .map(|s| s.to_string())
+                            .ok();
+                        Some(ScalarValue::Utf8(s))
+                    }
+                }
+            }
+            // type not supported yet
+            ParquetStatistics::FixedLenByteArray(s) => {
+                match $target_arrow_type {
+                    // just support the decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => None,
+                }
+            }
+        }
+    }};
+}
+
+#[derive(Debug, Clone, Copy)]
+enum Statistic {
+    Min,
+    Max,
+}
+
+/// Extracts statistics for a single leaf column from [`RowGroupMetaData`] as an
+/// arrow [`ArrayRef`]
+///
+/// For example, given a parquet file with 3 Row Groups, when asked for
+/// statistics for column "A" it will return a single array with 3 elements.
+pub(crate) struct RowGroupStatisticsConverter<'a> {
+    field: &'a Field,
+}
+
+impl<'a> RowGroupStatisticsConverter<'a> {
+    /// Create a new RowGoupStatisticsConverter for extracting the named column
+    /// in the schema.
+    pub fn try_new(schema: &'a arrow_schema::Schema, column_name: &str) -> Result<Self> {
+        let field = schema.field_with_name(column_name)?;
+        Ok(Self { field })
+    }
+
+    /// Returns the min value for the column into an array ref.
+    pub fn min<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(Statistic::Min, row_group_meta_data)
+    }
+
+    /// Returns the max value for the column into an array ref.
+    pub fn max<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(Statistic::Max, row_group_meta_data)
+    }
+
+    /// Extracts all min/max values for the column into an array ref.
+    fn min_max_impl<'b>(
+        &self,
+        mm: Statistic,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        let mut row_group_meta_data = row_group_meta_data.into_iter().peekable();
+
+        // if it is empty, return empty array
+        if row_group_meta_data.peek().is_none() {
+            return Ok(new_empty_array(self.field.data_type()));
+        }
+
+        let maybe_index = row_group_meta_data.peek().and_then(|rg_meta| {
+            rg_meta
+                .columns()
+                .iter()
+                .enumerate()
+                .find(|(_idx, c)| c.column_descr().name() == self.field.name())
+                .map(|(idx, _c)| idx)
+        });
+
+        // don't have this column, return an array of all NULLs
+        let Some(column_index) = maybe_index else {
+            let num_row_groups = row_group_meta_data.count();
+            let sv = ScalarValue::try_from(self.field.data_type())?;
+            return sv.to_array_of_size(num_row_groups);
+        };
+
+        let stats_iter = row_group_meta_data.map(move |row_group_meta_data| {
+            row_group_meta_data.column(column_index).statistics()
+        });
+
+        // this is the value to use when the statistics are not set
+        let null_value = ScalarValue::try_from(self.field.data_type())?;
+        match mm {
+            Statistic::Min => {
+                let values = stats_iter.map(|column_statistics| {
+                    column_statistics
+                        .and_then(|column_statistics| {
+                            get_statistic!(
+                                column_statistics,
+                                min,
+                                min_bytes,
+                                Some(self.field.data_type().clone())
+                            )
+                        })
+                        .unwrap_or_else(|| null_value.clone())
+                });
+                ScalarValue::iter_to_array(values)
+            }
+            Statistic::Max => {
+                let values = stats_iter.map(|column_statistics| {
+                    column_statistics
+                        .and_then(|column_statistics| {
+                            get_statistic!(
+                                column_statistics,
+                                max,
+                                max_bytes,
+                                Some(self.field.data_type().clone())
+                            )
+                        })
+                        .unwrap_or_else(|| null_value.clone())
+                });
+                ScalarValue::iter_to_array(values)
+            }
+        }
+    }
+}
+
+#[cfg(test)]
+mod test {
+    use super::*;
+    use arrow_array::{
+        new_null_array, Array, BinaryArray, BooleanArray, Decimal128Array, Float32Array,
+        Float64Array, Int32Array, Int64Array, RecordBatch, StringArray, StructArray,
+        TimestampNanosecondArray,
+    };
+    use arrow_schema::SchemaRef;
+    use bytes::Bytes;
+    use datafusion_common::test_util::parquet_test_data;
+    use parquet::arrow::arrow_reader::ArrowReaderBuilder;
+    use parquet::arrow::arrow_writer::ArrowWriter;
+    use parquet::file::metadata::ParquetMetaData;
+    use parquet::file::properties::{EnabledStatistics, WriterProperties};
+    use std::path::PathBuf;
+    use std::sync::Arc;
+
+    // TODO error cases (with parquet statistics that are mismatched in expected type)
+
+    #[test]
+    fn roundtrip_empty() {
+        let empty_bool_array = new_empty_array(&DataType::Boolean);
+        Test {
+            input: empty_bool_array.clone(),
+            expected_min: empty_bool_array.clone(),
+            expected_max: empty_bool_array.clone(),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_bool() {
+        Test {
+            input: bool_array([
+                // row group 1
+                Some(true),
+                None,
+                Some(true),
+                // row group 2
+                Some(true),
+                Some(false),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ]),
+            expected_min: bool_array([Some(true), Some(false), None]),
+            expected_max: bool_array([Some(true), Some(true), None]),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_int32() {
+        Test {
+            input: i32_array([
+                // row group 1
+                Some(1),
+                None,
+                Some(3),
+                // row group 2
+                Some(0),
+                Some(5),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ]),
+            expected_min: i32_array([Some(1), Some(0), None]),
+            expected_max: i32_array([Some(3), Some(5), None]),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_int64() {
+        Test {
+            input: i64_array([
+                // row group 1
+                Some(1),
+                None,
+                Some(3),
+                // row group 2
+                Some(0),
+                Some(5),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ]),
+            expected_min: i64_array([Some(1), Some(0), None]),
+            expected_max: i64_array(vec![Some(3), Some(5), None]),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_f32() {
+        Test {
+            input: f32_array([
+                // row group 1
+                Some(1.0),
+                None,
+                Some(3.0),
+                // row group 2
+                Some(-1.0),
+                Some(5.0),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ]),
+            expected_min: f32_array([Some(1.0), Some(-1.0), None]),
+            expected_max: f32_array([Some(3.0), Some(5.0), None]),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_f64() {
+        Test {
+            input: f64_array([
+                // row group 1
+                Some(1.0),
+                None,
+                Some(3.0),
+                // row group 2
+                Some(-1.0),
+                Some(5.0),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ]),
+            expected_min: f64_array([Some(1.0), Some(-1.0), None]),
+            expected_max: f64_array([Some(3.0), Some(5.0), None]),
+        }
+        .run()
+    }
+
+    #[test]
+    #[should_panic(
+        expected = "Inconsistent types in ScalarValue::iter_to_array. Expected Int64, got TimestampNanosecond(NULL, None)"
+    )]
+    // Due to https://github.com/apache/arrow-datafusion/issues/8295
+    fn roundtrip_timestamp() {
+        Test {
+            input: timestamp_array([
+                // row group 1
+                Some(1),
+                None,
+                Some(3),
+                // row group 2
+                Some(9),
+                Some(5),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ]),
+            expected_min: timestamp_array([Some(1), Some(5), None]),
+            expected_max: timestamp_array([Some(3), Some(9), None]),
+        }
+        .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]
+    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/arrow-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]
+    #[should_panic(
+        expected = "Inconsistent types in ScalarValue::iter_to_array. Expected Utf8, got Binary(NULL)"
+    )]
+    // Due to https://github.com/apache/arrow-datafusion/issues/8295
+    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 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)]);
+
+        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);
+
+        // read the int_col statistics
+        let (_idx, field) = schema.column_with_name("int_col").unwrap();
+
+        let converter =
+            RowGroupStatisticsConverter::try_new(&schema, field.name()).unwrap();
+        let row_groups = metadata.row_groups();
+        let min = converter.min(row_groups).unwrap();
+        assert_eq!(
+            &min,
+            &expected_min,
+            "Min. Statistics\n\n{}\n\n",
+            DisplayStats(row_groups)
+        );
+
+        let max = converter.max(row_groups).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: i32_array([Some(0)]),
+                expected_max: i32_array([Some(9)]),
+            })
+            .with_column(ExpectedColumn {
+                name: "smallint_col",
+                expected_min: i32_array([Some(0)]),
+                expected_max: i32_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_array([None]),
+                expected_max: timestamp_array([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);
+
+            for field in schema.fields() {
+                let converter =
+                    RowGroupStatisticsConverter::try_new(&schema, field.name()).unwrap();
+                let row_groups = metadata.row_groups();
+                let min = converter.min(row_groups).unwrap();
+                assert_eq!(
+                    &min,
+                    &expected_min,
+                    "Min. Statistics\n\n{}\n\n",
+                    DisplayStats(row_groups)
+                );
+
+                let max = converter.max(row_groups).unwrap();
+                assert_eq!(
+                    &max,
+                    &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<ParquetMetaData> {
+        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<ExpectedColumn>,
+    }
+
+    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 exepcted 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();
+
+            for expected_column in self.expected_columns {
+                let ExpectedColumn {
+                    name,
+                    expected_min,
+                    expected_max,
+                } = expected_column;
+
+                let converter = RowGroupStatisticsConverter::try_new(arrow_schema, name)
+                    .expect("can't find field in schema");
+
+                let actual_min = converter.min(metadata.row_groups()).unwrap();
+                assert_eq!(&expected_min, &actual_min, "column {name}");
+
+                let actual_max = converter.max(metadata.row_groups()).unwrap();
+                assert_eq!(&expected_max, &actual_max, "column {name}");
+            }
+        }
+    }
+
+    fn bool_array(input: impl IntoIterator<Item = Option<bool>>) -> ArrayRef {
+        let array: BooleanArray = input.into_iter().collect();
+        Arc::new(array)
+    }
+
+    fn i32_array(input: impl IntoIterator<Item = Option<i32>>) -> ArrayRef {
+        let array: Int32Array = input.into_iter().collect();
+        Arc::new(array)
+    }
+
+    fn i64_array(input: impl IntoIterator<Item = Option<i64>>) -> ArrayRef {
+        let array: Int64Array = input.into_iter().collect();
+        Arc::new(array)
+    }
+
+    fn f32_array(input: impl IntoIterator<Item = Option<f32>>) -> ArrayRef {
+        let array: Float32Array = input.into_iter().collect();
+        Arc::new(array)
+    }
+
+    fn f64_array(input: impl IntoIterator<Item = Option<f64>>) -> ArrayRef {
+        let array: Float64Array = input.into_iter().collect();
+        Arc::new(array)
+    }
+
+    fn timestamp_array(input: impl IntoIterator<Item = Option<i64>>) -> ArrayRef {
+        let array: TimestampNanosecondArray = input.into_iter().collect();
+        Arc::new(array)
+    }
+
+    fn utf8_array<'a>(input: impl IntoIterator<Item = Option<&'a str>>) -> 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 "b" and "i" with the specified values
+    fn struct_array(input: Vec<(Option<bool>, Option<i32>)>) -> 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("b", DataType::Boolean, nullable)),
+                Arc::new(boolean) as ArrayRef,
+            ),
+            (
+                Arc::new(Field::new("i", DataType::Int32, nullable)),

Review Comment:
   I filed https://github.com/apache/arrow-datafusion/issues/8335 for this issue



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Extract parquet statistics to its own module, add tests [arrow-datafusion]

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb merged PR #8294:
URL: https://github.com/apache/arrow-datafusion/pull/8294


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Extract parquet statistics to its own module, add tests [arrow-datafusion]

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on code in PR #8294:
URL: https://github.com/apache/arrow-datafusion/pull/8294#discussion_r1400984980


##########
datafusion/core/src/datasource/physical_plan/parquet/statistics.rs:
##########
@@ -0,0 +1,805 @@
+// 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::ArrayRef, datatypes::DataType};
+use arrow_array::new_empty_array;
+use arrow_schema::Field;
+use datafusion_common::{Result, ScalarValue};
+use parquet::file::{
+    metadata::RowGroupMetaData, statistics::Statistics as ParquetStatistics,
+};
+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(b))
+}
+
+// Copy from arrow-rs
+// https://github.com/apache/arrow-rs/blob/733b7e7fd1e8c43a404c3ce40ecf741d493c21b4/parquet/src/arrow/buffer/bit_util.rs#L55
+// Convert the byte slice to fixed length byte array with the length of 16
+fn sign_extend_be(b: &[u8]) -> [u8; 16] {
+    assert!(b.len() <= 16, "Array too large, expected less than 16");
+    let is_negative = (b[0] & 128u8) == 128u8;
+    let mut result = if is_negative { [255u8; 16] } else { [0u8; 16] };
+    for (d, s) in result.iter_mut().skip(16 - b.len()).zip(b) {
+        *d = *s;
+    }
+    result
+}
+
+/// Converts parquet RowGroup statistics (stored in
+/// [`RowGroupMetaData`]) into an arrow [`ArrayRef`]
+///
+/// For example, given a parquet file with 3 Row Groups, when asked for
+/// statistics for column "A" it will return a single array with 3 elements,
+///
+pub(crate) struct RowGoupStatisticsConverter<'a> {
+    field: &'a Field,
+}
+
+/// Extract a single min/max statistics from a [`ParquetStatistics`] object
+///
+/// * `$column_statistics` is the `ParquetStatistics` object
+/// * `$func is the function` (`min`/`max`) to call to get the value
+/// * `$bytes_func` is the function (`min_bytes`/`max_bytes`) to call to get the value as bytes
+/// * `$target_arrow_type` is the [`DataType`] of the target statistics
+macro_rules! get_statistic {
+    ($column_statistics:expr, $func:ident, $bytes_func:ident, $target_arrow_type:expr) => {{
+        if !$column_statistics.has_min_max_set() {
+            return None;
+        }
+        match $column_statistics {
+            ParquetStatistics::Boolean(s) => Some(ScalarValue::Boolean(Some(*s.$func()))),
+            ParquetStatistics::Int32(s) => {
+                match $target_arrow_type {
+                    // int32 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int32(Some(*s.$func()))),
+                }
+            }
+            ParquetStatistics::Int64(s) => {
+                match $target_arrow_type {
+                    // int64 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int64(Some(*s.$func()))),
+                }
+            }
+            // 96 bit ints not supported
+            ParquetStatistics::Int96(_) => None,
+            ParquetStatistics::Float(s) => Some(ScalarValue::Float32(Some(*s.$func()))),
+            ParquetStatistics::Double(s) => Some(ScalarValue::Float64(Some(*s.$func()))),
+            ParquetStatistics::ByteArray(s) => {
+                match $target_arrow_type {
+                    // decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => {
+                        let s = std::str::from_utf8(s.$bytes_func())
+                            .map(|s| s.to_string())
+                            .ok();
+                        Some(ScalarValue::Utf8(s))
+                    }
+                }
+            }
+            // type not supported yet
+            ParquetStatistics::FixedLenByteArray(s) => {
+                match $target_arrow_type {
+                    // just support the decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => None,
+                }
+            }
+        }
+    }};
+}
+
+#[derive(Debug, Clone, Copy)]
+enum MinMax {
+    Min,
+    Max,
+}
+
+impl<'a> RowGoupStatisticsConverter<'a> {
+    /// Create a new RowGoupStatisticsConverter suitable that can extract
+    /// statistics for the specified field
+    pub fn new(field: &'a Field) -> Self {
+        Self { field }
+    }
+
+    /// Returns the min value for the column into an array ref.
+    pub fn min<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(MinMax::Min, row_group_meta_data)
+    }
+
+    /// Returns the max value for the column into an array ref.
+    pub fn max<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(MinMax::Max, row_group_meta_data)
+    }
+
+    /// Extracts all min/max values for the column into an array ref.
+    fn min_max_impl<'b>(
+        &self,
+        mm: MinMax,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        let mut row_group_meta_data = row_group_meta_data.into_iter().peekable();
+
+        // if it is empty, return empty array

Review Comment:
   this handling of empty iterators is new, to support the new array ref interface



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Extract parquet statistics to its own module, add tests [arrow-datafusion]

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on code in PR #8294:
URL: https://github.com/apache/arrow-datafusion/pull/8294#discussion_r1407991303


##########
datafusion/core/src/datasource/physical_plan/parquet/statistics.rs:
##########
@@ -0,0 +1,807 @@
+// 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.
+
+//! [`RowGroupStatisticsConverter`] tp converts parquet RowGroup statistics to arrow [`ArrayRef`].
+
+use arrow::{array::ArrayRef, datatypes::DataType};
+use arrow_array::new_empty_array;
+use arrow_schema::Field;
+use datafusion_common::{Result, ScalarValue};
+use parquet::file::{
+    metadata::RowGroupMetaData, statistics::Statistics as ParquetStatistics,
+};
+
+// 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(b))
+}
+
+// Copy from arrow-rs
+// https://github.com/apache/arrow-rs/blob/733b7e7fd1e8c43a404c3ce40ecf741d493c21b4/parquet/src/arrow/buffer/bit_util.rs#L55
+// Convert the byte slice to fixed length byte array with the length of 16
+fn sign_extend_be(b: &[u8]) -> [u8; 16] {
+    assert!(b.len() <= 16, "Array too large, expected less than 16");
+    let is_negative = (b[0] & 128u8) == 128u8;
+    let mut result = if is_negative { [255u8; 16] } else { [0u8; 16] };
+    for (d, s) in result.iter_mut().skip(16 - b.len()).zip(b) {
+        *d = *s;
+    }
+    result
+}
+
+/// Converts parquet RowGroup statistics (stored in
+/// [`RowGroupMetaData`]) into an arrow [`ArrayRef`]
+///
+/// For example, given a parquet file with 3 Row Groups, when asked for
+/// statistics for column "A" it will return a single array with 3 elements,
+///
+pub(crate) struct RowGroupStatisticsConverter<'a> {
+    field: &'a Field,
+}
+
+/// Extract a single min/max statistics from a [`ParquetStatistics`] object
+///
+/// * `$column_statistics` is the `ParquetStatistics` object
+/// * `$func is the function` (`min`/`max`) to call to get the value
+/// * `$bytes_func` is the function (`min_bytes`/`max_bytes`) to call to get the value as bytes
+/// * `$target_arrow_type` is the [`DataType`] of the target statistics
+macro_rules! get_statistic {
+    ($column_statistics:expr, $func:ident, $bytes_func:ident, $target_arrow_type:expr) => {{
+        if !$column_statistics.has_min_max_set() {
+            return None;
+        }
+        match $column_statistics {
+            ParquetStatistics::Boolean(s) => Some(ScalarValue::Boolean(Some(*s.$func()))),
+            ParquetStatistics::Int32(s) => {
+                match $target_arrow_type {
+                    // int32 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int32(Some(*s.$func()))),
+                }
+            }
+            ParquetStatistics::Int64(s) => {
+                match $target_arrow_type {
+                    // int64 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int64(Some(*s.$func()))),
+                }
+            }
+            // 96 bit ints not supported
+            ParquetStatistics::Int96(_) => None,
+            ParquetStatistics::Float(s) => Some(ScalarValue::Float32(Some(*s.$func()))),
+            ParquetStatistics::Double(s) => Some(ScalarValue::Float64(Some(*s.$func()))),
+            ParquetStatistics::ByteArray(s) => {
+                match $target_arrow_type {
+                    // decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => {
+                        let s = std::str::from_utf8(s.$bytes_func())
+                            .map(|s| s.to_string())
+                            .ok();
+                        Some(ScalarValue::Utf8(s))
+                    }
+                }
+            }
+            // type not supported yet
+            ParquetStatistics::FixedLenByteArray(s) => {
+                match $target_arrow_type {
+                    // just support the decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => None,
+                }
+            }
+        }
+    }};
+}
+
+#[derive(Debug, Clone, Copy)]
+enum MinMax {
+    Min,
+    Max,
+}
+
+impl<'a> RowGroupStatisticsConverter<'a> {
+    /// Create a new RowGoupStatisticsConverter suitable that can extract
+    /// statistics for the specified field
+    pub fn new(field: &'a Field) -> Self {
+        Self { field }
+    }
+
+    /// Returns the min value for the column into an array ref.
+    pub fn min<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(MinMax::Min, row_group_meta_data)
+    }
+
+    /// Returns the max value for the column into an array ref.
+    pub fn max<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(MinMax::Max, row_group_meta_data)
+    }
+
+    /// Extracts all min/max values for the column into an array ref.
+    fn min_max_impl<'b>(
+        &self,
+        mm: MinMax,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        let mut row_group_meta_data = row_group_meta_data.into_iter().peekable();
+
+        // if it is empty, return empty array
+        if row_group_meta_data.peek().is_none() {
+            return Ok(new_empty_array(self.field.data_type()));
+        }
+
+        let maybe_index = row_group_meta_data.peek().and_then(|rg_meta| {
+            rg_meta
+                .columns()
+                .iter()
+                .enumerate()
+                .find(|(_idx, c)| c.column_descr().name() == self.field.name())
+                .map(|(idx, _c)| idx)
+        });
+
+        // don't have this column, return an array of all NULLs
+        let Some(column_index) = maybe_index else {
+            let num_row_groups = row_group_meta_data.count();
+            let sv = ScalarValue::try_from(self.field.data_type())?;
+            return sv.to_array_of_size(num_row_groups);
+        };
+
+        let stats_iter = row_group_meta_data.map(move |row_group_meta_data| {
+            row_group_meta_data.column(column_index).statistics()
+        });
+
+        // this is the value to use when the statistics are not set
+        let null_value = ScalarValue::try_from(self.field.data_type())?;
+        match mm {
+            MinMax::Min => {
+                let values = stats_iter.map(|column_statistics| {
+                    column_statistics
+                        .and_then(|column_statistics| {
+                            get_statistic!(
+                                column_statistics,
+                                min,
+                                min_bytes,
+                                Some(self.field.data_type().clone())
+                            )
+                        })
+                        .unwrap_or_else(|| null_value.clone())
+                });
+                ScalarValue::iter_to_array(values)
+            }
+            MinMax::Max => {
+                let values = stats_iter.map(|column_statistics| {
+                    column_statistics
+                        .and_then(|column_statistics| {
+                            get_statistic!(
+                                column_statistics,
+                                max,
+                                max_bytes,
+                                Some(self.field.data_type().clone())
+                            )
+                        })
+                        .unwrap_or_else(|| null_value.clone())
+                });
+                ScalarValue::iter_to_array(values)
+            }
+        }
+    }
+}
+
+#[cfg(test)]
+mod test {
+    use super::*;
+    use arrow_array::{
+        BinaryArray, BooleanArray, Decimal128Array, Float32Array, Float64Array,
+        Int32Array, Int64Array, RecordBatch, StringArray, TimestampNanosecondArray,
+    };
+    use arrow_schema::SchemaRef;
+    use bytes::Bytes;
+    use datafusion_common::test_util::parquet_test_data;
+    use parquet::arrow::arrow_reader::ArrowReaderBuilder;
+    use parquet::arrow::arrow_writer::ArrowWriter;
+    use parquet::file::metadata::ParquetMetaData;
+    use parquet::file::properties::{EnabledStatistics, WriterProperties};
+    use std::path::PathBuf;
+    use std::sync::Arc;
+
+    // TODO error cases (with parquet statistics that are mismatched in expected type)
+

Review Comment:
   Update: we have added tests that now ensure the struct array statistics are read as NULL and that statistics are not incorrectly read from sub fields in structs



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Extract parquet statistics to its own module, add tests [arrow-datafusion]

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on code in PR #8294:
URL: https://github.com/apache/arrow-datafusion/pull/8294#discussion_r1406748077


##########
datafusion/core/src/datasource/physical_plan/parquet/statistics.rs:
##########
@@ -0,0 +1,925 @@
+// 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.
+
+//! [`RowGroupStatisticsConverter`] tp converts parquet RowGroup statistics to arrow [`ArrayRef`].
+
+use arrow::{array::ArrayRef, datatypes::DataType};
+use arrow_array::new_empty_array;
+use arrow_schema::Field;
+use datafusion_common::{Result, ScalarValue};
+use parquet::file::{
+    metadata::RowGroupMetaData, statistics::Statistics as ParquetStatistics,
+};
+
+// 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(b))
+}
+
+// Copy from arrow-rs
+// https://github.com/apache/arrow-rs/blob/733b7e7fd1e8c43a404c3ce40ecf741d493c21b4/parquet/src/arrow/buffer/bit_util.rs#L55
+// Convert the byte slice to fixed length byte array with the length of 16
+fn sign_extend_be(b: &[u8]) -> [u8; 16] {
+    assert!(b.len() <= 16, "Array too large, expected less than 16");
+    let is_negative = (b[0] & 128u8) == 128u8;
+    let mut result = if is_negative { [255u8; 16] } else { [0u8; 16] };
+    for (d, s) in result.iter_mut().skip(16 - b.len()).zip(b) {
+        *d = *s;
+    }
+    result
+}
+
+/// Extract a single min/max statistics from a [`ParquetStatistics`] object
+///
+/// * `$column_statistics` is the `ParquetStatistics` object
+/// * `$func is the function` (`min`/`max`) to call to get the value
+/// * `$bytes_func` is the function (`min_bytes`/`max_bytes`) to call to get the value as bytes
+/// * `$target_arrow_type` is the [`DataType`] of the target statistics
+macro_rules! get_statistic {
+    ($column_statistics:expr, $func:ident, $bytes_func:ident, $target_arrow_type:expr) => {{
+        if !$column_statistics.has_min_max_set() {
+            return None;
+        }
+        match $column_statistics {
+            ParquetStatistics::Boolean(s) => Some(ScalarValue::Boolean(Some(*s.$func()))),
+            ParquetStatistics::Int32(s) => {
+                match $target_arrow_type {
+                    // int32 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int32(Some(*s.$func()))),
+                }
+            }
+            ParquetStatistics::Int64(s) => {
+                match $target_arrow_type {
+                    // int64 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int64(Some(*s.$func()))),
+                }
+            }
+            // 96 bit ints not supported
+            ParquetStatistics::Int96(_) => None,
+            ParquetStatistics::Float(s) => Some(ScalarValue::Float32(Some(*s.$func()))),
+            ParquetStatistics::Double(s) => Some(ScalarValue::Float64(Some(*s.$func()))),
+            ParquetStatistics::ByteArray(s) => {
+                match $target_arrow_type {
+                    // decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => {
+                        let s = std::str::from_utf8(s.$bytes_func())
+                            .map(|s| s.to_string())
+                            .ok();
+                        Some(ScalarValue::Utf8(s))
+                    }
+                }
+            }
+            // type not supported yet
+            ParquetStatistics::FixedLenByteArray(s) => {
+                match $target_arrow_type {
+                    // just support the decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => None,
+                }
+            }
+        }
+    }};
+}
+
+#[derive(Debug, Clone, Copy)]
+enum Statistic {
+    Min,
+    Max,
+}
+
+/// Extracts statistics for a single leaf column from [`RowGroupMetaData`] as an
+/// arrow [`ArrayRef`]
+///
+/// For example, given a parquet file with 3 Row Groups, when asked for
+/// statistics for column "A" it will return a single array with 3 elements.
+pub(crate) struct RowGroupStatisticsConverter<'a> {
+    field: &'a Field,
+}
+
+impl<'a> RowGroupStatisticsConverter<'a> {
+    /// Create a new RowGoupStatisticsConverter for extracting the named column
+    /// in the schema.
+    pub fn try_new(schema: &'a arrow_schema::Schema, column_name: &str) -> Result<Self> {
+        let field = schema.field_with_name(column_name)?;
+        Ok(Self { field })
+    }
+
+    /// Returns the min value for the column into an array ref.
+    pub fn min<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(Statistic::Min, row_group_meta_data)
+    }
+
+    /// Returns the max value for the column into an array ref.
+    pub fn max<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(Statistic::Max, row_group_meta_data)
+    }
+
+    /// Extracts all min/max values for the column into an array ref.
+    fn min_max_impl<'b>(
+        &self,
+        mm: Statistic,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        let mut row_group_meta_data = row_group_meta_data.into_iter().peekable();
+
+        // if it is empty, return empty array
+        if row_group_meta_data.peek().is_none() {
+            return Ok(new_empty_array(self.field.data_type()));
+        }
+
+        let maybe_index = row_group_meta_data.peek().and_then(|rg_meta| {
+            rg_meta
+                .columns()
+                .iter()
+                .enumerate()
+                .find(|(_idx, c)| c.column_descr().name() == self.field.name())
+                .map(|(idx, _c)| idx)
+        });
+
+        // don't have this column, return an array of all NULLs
+        let Some(column_index) = maybe_index else {
+            let num_row_groups = row_group_meta_data.count();
+            let sv = ScalarValue::try_from(self.field.data_type())?;
+            return sv.to_array_of_size(num_row_groups);
+        };
+
+        let stats_iter = row_group_meta_data.map(move |row_group_meta_data| {
+            row_group_meta_data.column(column_index).statistics()
+        });
+
+        // this is the value to use when the statistics are not set
+        let null_value = ScalarValue::try_from(self.field.data_type())?;
+        match mm {
+            Statistic::Min => {
+                let values = stats_iter.map(|column_statistics| {
+                    column_statistics
+                        .and_then(|column_statistics| {
+                            get_statistic!(
+                                column_statistics,
+                                min,
+                                min_bytes,
+                                Some(self.field.data_type().clone())
+                            )
+                        })
+                        .unwrap_or_else(|| null_value.clone())
+                });
+                ScalarValue::iter_to_array(values)
+            }
+            Statistic::Max => {
+                let values = stats_iter.map(|column_statistics| {
+                    column_statistics
+                        .and_then(|column_statistics| {
+                            get_statistic!(
+                                column_statistics,
+                                max,
+                                max_bytes,
+                                Some(self.field.data_type().clone())
+                            )
+                        })
+                        .unwrap_or_else(|| null_value.clone())
+                });
+                ScalarValue::iter_to_array(values)
+            }
+        }
+    }
+}
+
+#[cfg(test)]
+mod test {
+    use super::*;
+    use arrow_array::{
+        new_null_array, Array, BinaryArray, BooleanArray, Decimal128Array, Float32Array,
+        Float64Array, Int32Array, Int64Array, RecordBatch, StringArray, StructArray,
+        TimestampNanosecondArray,
+    };
+    use arrow_schema::SchemaRef;
+    use bytes::Bytes;
+    use datafusion_common::test_util::parquet_test_data;
+    use parquet::arrow::arrow_reader::ArrowReaderBuilder;
+    use parquet::arrow::arrow_writer::ArrowWriter;
+    use parquet::file::metadata::ParquetMetaData;
+    use parquet::file::properties::{EnabledStatistics, WriterProperties};
+    use std::path::PathBuf;
+    use std::sync::Arc;
+
+    // TODO error cases (with parquet statistics that are mismatched in expected type)
+
+    #[test]
+    fn roundtrip_empty() {
+        let empty_bool_array = new_empty_array(&DataType::Boolean);
+        Test {
+            input: empty_bool_array.clone(),
+            expected_min: empty_bool_array.clone(),
+            expected_max: empty_bool_array.clone(),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_bool() {
+        Test {
+            input: bool_array([
+                // row group 1
+                Some(true),
+                None,
+                Some(true),
+                // row group 2
+                Some(true),
+                Some(false),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ]),
+            expected_min: bool_array([Some(true), Some(false), None]),
+            expected_max: bool_array([Some(true), Some(true), None]),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_int32() {
+        Test {
+            input: i32_array([
+                // row group 1
+                Some(1),
+                None,
+                Some(3),
+                // row group 2
+                Some(0),
+                Some(5),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ]),
+            expected_min: i32_array([Some(1), Some(0), None]),
+            expected_max: i32_array([Some(3), Some(5), None]),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_int64() {
+        Test {
+            input: i64_array([
+                // row group 1
+                Some(1),
+                None,
+                Some(3),
+                // row group 2
+                Some(0),
+                Some(5),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ]),
+            expected_min: i64_array([Some(1), Some(0), None]),
+            expected_max: i64_array(vec![Some(3), Some(5), None]),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_f32() {
+        Test {
+            input: f32_array([
+                // row group 1
+                Some(1.0),
+                None,
+                Some(3.0),
+                // row group 2
+                Some(-1.0),
+                Some(5.0),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ]),
+            expected_min: f32_array([Some(1.0), Some(-1.0), None]),
+            expected_max: f32_array([Some(3.0), Some(5.0), None]),
+        }
+        .run()
+    }
+
+    #[test]
+    fn roundtrip_f64() {
+        Test {
+            input: f64_array([
+                // row group 1
+                Some(1.0),
+                None,
+                Some(3.0),
+                // row group 2
+                Some(-1.0),
+                Some(5.0),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ]),
+            expected_min: f64_array([Some(1.0), Some(-1.0), None]),
+            expected_max: f64_array([Some(3.0), Some(5.0), None]),
+        }
+        .run()
+    }
+
+    #[test]
+    #[should_panic(
+        expected = "Inconsistent types in ScalarValue::iter_to_array. Expected Int64, got TimestampNanosecond(NULL, None)"
+    )]
+    // Due to https://github.com/apache/arrow-datafusion/issues/8295
+    fn roundtrip_timestamp() {
+        Test {
+            input: timestamp_array([
+                // row group 1
+                Some(1),
+                None,
+                Some(3),
+                // row group 2
+                Some(9),
+                Some(5),
+                None,
+                // row group 3
+                None,
+                None,
+                None,
+            ]),
+            expected_min: timestamp_array([Some(1), Some(5), None]),
+            expected_max: timestamp_array([Some(3), Some(9), None]),
+        }
+        .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]
+    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 {

Review Comment:
   Unless I am mistaken this is only working because the name of the struct array happens to not collide with that of any of the leaves, if this wasn't the case you would project a random column



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Extract parquet statistics to its own module, add tests [arrow-datafusion]

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on code in PR #8294:
URL: https://github.com/apache/arrow-datafusion/pull/8294#discussion_r1406585965


##########
datafusion/core/src/datasource/physical_plan/parquet/statistics.rs:
##########
@@ -0,0 +1,807 @@
+// 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.
+
+//! [`RowGroupStatisticsConverter`] tp converts parquet RowGroup statistics to arrow [`ArrayRef`].
+
+use arrow::{array::ArrayRef, datatypes::DataType};
+use arrow_array::new_empty_array;
+use arrow_schema::Field;
+use datafusion_common::{Result, ScalarValue};
+use parquet::file::{
+    metadata::RowGroupMetaData, statistics::Statistics as ParquetStatistics,
+};
+
+// 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(b))
+}
+
+// Copy from arrow-rs
+// https://github.com/apache/arrow-rs/blob/733b7e7fd1e8c43a404c3ce40ecf741d493c21b4/parquet/src/arrow/buffer/bit_util.rs#L55
+// Convert the byte slice to fixed length byte array with the length of 16
+fn sign_extend_be(b: &[u8]) -> [u8; 16] {
+    assert!(b.len() <= 16, "Array too large, expected less than 16");
+    let is_negative = (b[0] & 128u8) == 128u8;
+    let mut result = if is_negative { [255u8; 16] } else { [0u8; 16] };
+    for (d, s) in result.iter_mut().skip(16 - b.len()).zip(b) {
+        *d = *s;
+    }
+    result
+}
+
+/// Converts parquet RowGroup statistics (stored in
+/// [`RowGroupMetaData`]) into an arrow [`ArrayRef`]
+///
+/// For example, given a parquet file with 3 Row Groups, when asked for
+/// statistics for column "A" it will return a single array with 3 elements,
+///
+pub(crate) struct RowGroupStatisticsConverter<'a> {
+    field: &'a Field,
+}
+
+/// Extract a single min/max statistics from a [`ParquetStatistics`] object
+///
+/// * `$column_statistics` is the `ParquetStatistics` object
+/// * `$func is the function` (`min`/`max`) to call to get the value
+/// * `$bytes_func` is the function (`min_bytes`/`max_bytes`) to call to get the value as bytes
+/// * `$target_arrow_type` is the [`DataType`] of the target statistics
+macro_rules! get_statistic {
+    ($column_statistics:expr, $func:ident, $bytes_func:ident, $target_arrow_type:expr) => {{
+        if !$column_statistics.has_min_max_set() {
+            return None;
+        }
+        match $column_statistics {
+            ParquetStatistics::Boolean(s) => Some(ScalarValue::Boolean(Some(*s.$func()))),
+            ParquetStatistics::Int32(s) => {
+                match $target_arrow_type {
+                    // int32 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int32(Some(*s.$func()))),
+                }
+            }
+            ParquetStatistics::Int64(s) => {
+                match $target_arrow_type {
+                    // int64 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int64(Some(*s.$func()))),
+                }
+            }
+            // 96 bit ints not supported
+            ParquetStatistics::Int96(_) => None,
+            ParquetStatistics::Float(s) => Some(ScalarValue::Float32(Some(*s.$func()))),
+            ParquetStatistics::Double(s) => Some(ScalarValue::Float64(Some(*s.$func()))),
+            ParquetStatistics::ByteArray(s) => {
+                match $target_arrow_type {
+                    // decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => {
+                        let s = std::str::from_utf8(s.$bytes_func())
+                            .map(|s| s.to_string())
+                            .ok();
+                        Some(ScalarValue::Utf8(s))
+                    }
+                }
+            }
+            // type not supported yet
+            ParquetStatistics::FixedLenByteArray(s) => {
+                match $target_arrow_type {
+                    // just support the decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => None,
+                }
+            }
+        }
+    }};
+}
+
+#[derive(Debug, Clone, Copy)]
+enum MinMax {
+    Min,
+    Max,
+}
+
+impl<'a> RowGroupStatisticsConverter<'a> {
+    /// Create a new RowGoupStatisticsConverter suitable that can extract
+    /// statistics for the specified field
+    pub fn new(field: &'a Field) -> Self {
+        Self { field }
+    }
+
+    /// Returns the min value for the column into an array ref.
+    pub fn min<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(MinMax::Min, row_group_meta_data)
+    }
+
+    /// Returns the max value for the column into an array ref.
+    pub fn max<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(MinMax::Max, row_group_meta_data)
+    }
+
+    /// Extracts all min/max values for the column into an array ref.
+    fn min_max_impl<'b>(
+        &self,
+        mm: MinMax,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        let mut row_group_meta_data = row_group_meta_data.into_iter().peekable();
+
+        // if it is empty, return empty array
+        if row_group_meta_data.peek().is_none() {
+            return Ok(new_empty_array(self.field.data_type()));
+        }
+
+        let maybe_index = row_group_meta_data.peek().and_then(|rg_meta| {
+            rg_meta
+                .columns()
+                .iter()
+                .enumerate()
+                .find(|(_idx, c)| c.column_descr().name() == self.field.name())
+                .map(|(idx, _c)| idx)
+        });
+
+        // don't have this column, return an array of all NULLs
+        let Some(column_index) = maybe_index else {
+            let num_row_groups = row_group_meta_data.count();
+            let sv = ScalarValue::try_from(self.field.data_type())?;
+            return sv.to_array_of_size(num_row_groups);
+        };
+
+        let stats_iter = row_group_meta_data.map(move |row_group_meta_data| {
+            row_group_meta_data.column(column_index).statistics()
+        });
+
+        // this is the value to use when the statistics are not set
+        let null_value = ScalarValue::try_from(self.field.data_type())?;
+        match mm {
+            MinMax::Min => {
+                let values = stats_iter.map(|column_statistics| {
+                    column_statistics
+                        .and_then(|column_statistics| {
+                            get_statistic!(
+                                column_statistics,
+                                min,
+                                min_bytes,
+                                Some(self.field.data_type().clone())
+                            )
+                        })
+                        .unwrap_or_else(|| null_value.clone())
+                });
+                ScalarValue::iter_to_array(values)
+            }
+            MinMax::Max => {
+                let values = stats_iter.map(|column_statistics| {
+                    column_statistics
+                        .and_then(|column_statistics| {
+                            get_statistic!(
+                                column_statistics,
+                                max,
+                                max_bytes,
+                                Some(self.field.data_type().clone())
+                            )
+                        })
+                        .unwrap_or_else(|| null_value.clone())
+                });
+                ScalarValue::iter_to_array(values)

Review Comment:
   Yes, exactly -- I plan to fix the code to build arrays directly, but I want to keep the current logic and add good test coverage before I do so so I can avoid regressions



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Extract parquet statistics to its own module, add tests [arrow-datafusion]

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on code in PR #8294:
URL: https://github.com/apache/arrow-datafusion/pull/8294#discussion_r1402762958


##########
datafusion/core/src/datasource/physical_plan/parquet/statistics.rs:
##########
@@ -0,0 +1,807 @@
+// 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.
+
+//! [`RowGroupStatisticsConverter`] tp converts parquet RowGroup statistics to arrow [`ArrayRef`].
+
+use arrow::{array::ArrayRef, datatypes::DataType};
+use arrow_array::new_empty_array;
+use arrow_schema::Field;
+use datafusion_common::{Result, ScalarValue};
+use parquet::file::{
+    metadata::RowGroupMetaData, statistics::Statistics as ParquetStatistics,
+};
+
+// 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(b))
+}
+
+// Copy from arrow-rs
+// https://github.com/apache/arrow-rs/blob/733b7e7fd1e8c43a404c3ce40ecf741d493c21b4/parquet/src/arrow/buffer/bit_util.rs#L55
+// Convert the byte slice to fixed length byte array with the length of 16
+fn sign_extend_be(b: &[u8]) -> [u8; 16] {
+    assert!(b.len() <= 16, "Array too large, expected less than 16");
+    let is_negative = (b[0] & 128u8) == 128u8;
+    let mut result = if is_negative { [255u8; 16] } else { [0u8; 16] };
+    for (d, s) in result.iter_mut().skip(16 - b.len()).zip(b) {
+        *d = *s;
+    }
+    result
+}
+
+/// Converts parquet RowGroup statistics (stored in
+/// [`RowGroupMetaData`]) into an arrow [`ArrayRef`]

Review Comment:
   ```suggestion
   /// Extracts statistics for a single leaf column from [`RowGroupMetaData`] as an arrow [`ArrayRef`]
   ```



##########
datafusion/core/src/datasource/physical_plan/parquet/statistics.rs:
##########
@@ -0,0 +1,807 @@
+// 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.
+
+//! [`RowGroupStatisticsConverter`] tp converts parquet RowGroup statistics to arrow [`ArrayRef`].
+
+use arrow::{array::ArrayRef, datatypes::DataType};
+use arrow_array::new_empty_array;
+use arrow_schema::Field;
+use datafusion_common::{Result, ScalarValue};
+use parquet::file::{
+    metadata::RowGroupMetaData, statistics::Statistics as ParquetStatistics,
+};
+
+// 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(b))
+}
+
+// Copy from arrow-rs
+// https://github.com/apache/arrow-rs/blob/733b7e7fd1e8c43a404c3ce40ecf741d493c21b4/parquet/src/arrow/buffer/bit_util.rs#L55
+// Convert the byte slice to fixed length byte array with the length of 16
+fn sign_extend_be(b: &[u8]) -> [u8; 16] {
+    assert!(b.len() <= 16, "Array too large, expected less than 16");
+    let is_negative = (b[0] & 128u8) == 128u8;
+    let mut result = if is_negative { [255u8; 16] } else { [0u8; 16] };
+    for (d, s) in result.iter_mut().skip(16 - b.len()).zip(b) {
+        *d = *s;
+    }
+    result
+}
+
+/// Converts parquet RowGroup statistics (stored in
+/// [`RowGroupMetaData`]) into an arrow [`ArrayRef`]
+///
+/// For example, given a parquet file with 3 Row Groups, when asked for
+/// statistics for column "A" it will return a single array with 3 elements,
+///
+pub(crate) struct RowGroupStatisticsConverter<'a> {
+    field: &'a Field,
+}
+
+/// Extract a single min/max statistics from a [`ParquetStatistics`] object
+///
+/// * `$column_statistics` is the `ParquetStatistics` object
+/// * `$func is the function` (`min`/`max`) to call to get the value
+/// * `$bytes_func` is the function (`min_bytes`/`max_bytes`) to call to get the value as bytes
+/// * `$target_arrow_type` is the [`DataType`] of the target statistics
+macro_rules! get_statistic {
+    ($column_statistics:expr, $func:ident, $bytes_func:ident, $target_arrow_type:expr) => {{
+        if !$column_statistics.has_min_max_set() {
+            return None;
+        }
+        match $column_statistics {
+            ParquetStatistics::Boolean(s) => Some(ScalarValue::Boolean(Some(*s.$func()))),
+            ParquetStatistics::Int32(s) => {
+                match $target_arrow_type {
+                    // int32 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int32(Some(*s.$func()))),
+                }
+            }
+            ParquetStatistics::Int64(s) => {
+                match $target_arrow_type {
+                    // int64 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int64(Some(*s.$func()))),
+                }
+            }
+            // 96 bit ints not supported
+            ParquetStatistics::Int96(_) => None,
+            ParquetStatistics::Float(s) => Some(ScalarValue::Float32(Some(*s.$func()))),
+            ParquetStatistics::Double(s) => Some(ScalarValue::Float64(Some(*s.$func()))),
+            ParquetStatistics::ByteArray(s) => {
+                match $target_arrow_type {
+                    // decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => {
+                        let s = std::str::from_utf8(s.$bytes_func())
+                            .map(|s| s.to_string())
+                            .ok();
+                        Some(ScalarValue::Utf8(s))
+                    }
+                }
+            }
+            // type not supported yet
+            ParquetStatistics::FixedLenByteArray(s) => {
+                match $target_arrow_type {
+                    // just support the decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => None,
+                }
+            }
+        }
+    }};
+}
+
+#[derive(Debug, Clone, Copy)]
+enum MinMax {

Review Comment:
   ```suggestion
   enum Statistic {
   ```



##########
datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs:
##########
@@ -431,11 +320,29 @@ macro_rules! get_null_count_values {
 
 impl<'a> PruningStatistics for RowGroupPruningStatistics<'a> {
     fn min_values(&self, column: &Column) -> Option<ArrayRef> {
-        get_min_max_values!(self, column, min, min_bytes)
+        let field = self
+            .parquet_schema
+            .fields()
+            .find(&column.name)
+            .map(|(_idx, field)| field)?;
+
+        RowGroupStatisticsConverter::new(field)

Review Comment:
   There is a slight mismatch here as parquet handles schema nesting differently from arrow
   
   I'm not sure how `Column` addresses nested fields, but I would expect to see something walking SchemaDescriptor to compute this mapping, or something similar.



##########
datafusion/core/src/datasource/physical_plan/parquet/statistics.rs:
##########
@@ -0,0 +1,807 @@
+// 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.
+
+//! [`RowGroupStatisticsConverter`] tp converts parquet RowGroup statistics to arrow [`ArrayRef`].
+
+use arrow::{array::ArrayRef, datatypes::DataType};
+use arrow_array::new_empty_array;
+use arrow_schema::Field;
+use datafusion_common::{Result, ScalarValue};
+use parquet::file::{
+    metadata::RowGroupMetaData, statistics::Statistics as ParquetStatistics,
+};
+
+// 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(b))
+}
+
+// Copy from arrow-rs
+// https://github.com/apache/arrow-rs/blob/733b7e7fd1e8c43a404c3ce40ecf741d493c21b4/parquet/src/arrow/buffer/bit_util.rs#L55
+// Convert the byte slice to fixed length byte array with the length of 16
+fn sign_extend_be(b: &[u8]) -> [u8; 16] {
+    assert!(b.len() <= 16, "Array too large, expected less than 16");
+    let is_negative = (b[0] & 128u8) == 128u8;
+    let mut result = if is_negative { [255u8; 16] } else { [0u8; 16] };
+    for (d, s) in result.iter_mut().skip(16 - b.len()).zip(b) {
+        *d = *s;
+    }
+    result
+}
+
+/// Converts parquet RowGroup statistics (stored in
+/// [`RowGroupMetaData`]) into an arrow [`ArrayRef`]
+///
+/// For example, given a parquet file with 3 Row Groups, when asked for
+/// statistics for column "A" it will return a single array with 3 elements,
+///
+pub(crate) struct RowGroupStatisticsConverter<'a> {
+    field: &'a Field,
+}
+
+/// Extract a single min/max statistics from a [`ParquetStatistics`] object
+///
+/// * `$column_statistics` is the `ParquetStatistics` object
+/// * `$func is the function` (`min`/`max`) to call to get the value
+/// * `$bytes_func` is the function (`min_bytes`/`max_bytes`) to call to get the value as bytes
+/// * `$target_arrow_type` is the [`DataType`] of the target statistics
+macro_rules! get_statistic {
+    ($column_statistics:expr, $func:ident, $bytes_func:ident, $target_arrow_type:expr) => {{
+        if !$column_statistics.has_min_max_set() {
+            return None;
+        }
+        match $column_statistics {
+            ParquetStatistics::Boolean(s) => Some(ScalarValue::Boolean(Some(*s.$func()))),
+            ParquetStatistics::Int32(s) => {
+                match $target_arrow_type {
+                    // int32 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int32(Some(*s.$func()))),
+                }
+            }
+            ParquetStatistics::Int64(s) => {
+                match $target_arrow_type {
+                    // int64 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int64(Some(*s.$func()))),
+                }
+            }
+            // 96 bit ints not supported
+            ParquetStatistics::Int96(_) => None,
+            ParquetStatistics::Float(s) => Some(ScalarValue::Float32(Some(*s.$func()))),
+            ParquetStatistics::Double(s) => Some(ScalarValue::Float64(Some(*s.$func()))),
+            ParquetStatistics::ByteArray(s) => {
+                match $target_arrow_type {
+                    // decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => {
+                        let s = std::str::from_utf8(s.$bytes_func())
+                            .map(|s| s.to_string())
+                            .ok();
+                        Some(ScalarValue::Utf8(s))
+                    }
+                }
+            }
+            // type not supported yet
+            ParquetStatistics::FixedLenByteArray(s) => {
+                match $target_arrow_type {
+                    // just support the decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => None,
+                }
+            }
+        }
+    }};
+}
+
+#[derive(Debug, Clone, Copy)]
+enum MinMax {
+    Min,
+    Max,
+}
+
+impl<'a> RowGroupStatisticsConverter<'a> {

Review Comment:
   I think this should either handle all columns, i.e. Schema / RowGroupMetadata or a single leaf column i.e. Field / ColumnChunkMetaData. I think of these options the latter is more flexible



##########
datafusion/core/src/datasource/physical_plan/parquet/statistics.rs:
##########
@@ -0,0 +1,807 @@
+// 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.
+
+//! [`RowGroupStatisticsConverter`] tp converts parquet RowGroup statistics to arrow [`ArrayRef`].
+
+use arrow::{array::ArrayRef, datatypes::DataType};
+use arrow_array::new_empty_array;
+use arrow_schema::Field;
+use datafusion_common::{Result, ScalarValue};
+use parquet::file::{
+    metadata::RowGroupMetaData, statistics::Statistics as ParquetStatistics,
+};
+
+// 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(b))
+}
+
+// Copy from arrow-rs
+// https://github.com/apache/arrow-rs/blob/733b7e7fd1e8c43a404c3ce40ecf741d493c21b4/parquet/src/arrow/buffer/bit_util.rs#L55
+// Convert the byte slice to fixed length byte array with the length of 16
+fn sign_extend_be(b: &[u8]) -> [u8; 16] {
+    assert!(b.len() <= 16, "Array too large, expected less than 16");
+    let is_negative = (b[0] & 128u8) == 128u8;
+    let mut result = if is_negative { [255u8; 16] } else { [0u8; 16] };
+    for (d, s) in result.iter_mut().skip(16 - b.len()).zip(b) {
+        *d = *s;
+    }
+    result
+}
+
+/// Converts parquet RowGroup statistics (stored in
+/// [`RowGroupMetaData`]) into an arrow [`ArrayRef`]
+///
+/// For example, given a parquet file with 3 Row Groups, when asked for
+/// statistics for column "A" it will return a single array with 3 elements,
+///
+pub(crate) struct RowGroupStatisticsConverter<'a> {
+    field: &'a Field,
+}
+
+/// Extract a single min/max statistics from a [`ParquetStatistics`] object
+///
+/// * `$column_statistics` is the `ParquetStatistics` object
+/// * `$func is the function` (`min`/`max`) to call to get the value
+/// * `$bytes_func` is the function (`min_bytes`/`max_bytes`) to call to get the value as bytes
+/// * `$target_arrow_type` is the [`DataType`] of the target statistics
+macro_rules! get_statistic {
+    ($column_statistics:expr, $func:ident, $bytes_func:ident, $target_arrow_type:expr) => {{
+        if !$column_statistics.has_min_max_set() {
+            return None;
+        }
+        match $column_statistics {
+            ParquetStatistics::Boolean(s) => Some(ScalarValue::Boolean(Some(*s.$func()))),
+            ParquetStatistics::Int32(s) => {
+                match $target_arrow_type {
+                    // int32 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int32(Some(*s.$func()))),
+                }
+            }
+            ParquetStatistics::Int64(s) => {
+                match $target_arrow_type {
+                    // int64 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int64(Some(*s.$func()))),
+                }
+            }
+            // 96 bit ints not supported
+            ParquetStatistics::Int96(_) => None,
+            ParquetStatistics::Float(s) => Some(ScalarValue::Float32(Some(*s.$func()))),
+            ParquetStatistics::Double(s) => Some(ScalarValue::Float64(Some(*s.$func()))),
+            ParquetStatistics::ByteArray(s) => {
+                match $target_arrow_type {
+                    // decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => {
+                        let s = std::str::from_utf8(s.$bytes_func())
+                            .map(|s| s.to_string())
+                            .ok();
+                        Some(ScalarValue::Utf8(s))
+                    }
+                }
+            }
+            // type not supported yet
+            ParquetStatistics::FixedLenByteArray(s) => {
+                match $target_arrow_type {
+                    // just support the decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => None,
+                }
+            }
+        }
+    }};
+}
+
+#[derive(Debug, Clone, Copy)]
+enum MinMax {
+    Min,
+    Max,
+}
+
+impl<'a> RowGroupStatisticsConverter<'a> {
+    /// Create a new RowGoupStatisticsConverter suitable that can extract
+    /// statistics for the specified field
+    pub fn new(field: &'a Field) -> Self {
+        Self { field }
+    }
+
+    /// Returns the min value for the column into an array ref.
+    pub fn min<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(MinMax::Min, row_group_meta_data)
+    }
+
+    /// Returns the max value for the column into an array ref.
+    pub fn max<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(MinMax::Max, row_group_meta_data)
+    }
+
+    /// Extracts all min/max values for the column into an array ref.
+    fn min_max_impl<'b>(
+        &self,
+        mm: MinMax,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        let mut row_group_meta_data = row_group_meta_data.into_iter().peekable();
+
+        // if it is empty, return empty array
+        if row_group_meta_data.peek().is_none() {
+            return Ok(new_empty_array(self.field.data_type()));
+        }
+
+        let maybe_index = row_group_meta_data.peek().and_then(|rg_meta| {
+            rg_meta
+                .columns()
+                .iter()
+                .enumerate()
+                .find(|(_idx, c)| c.column_descr().name() == self.field.name())
+                .map(|(idx, _c)| idx)
+        });
+
+        // don't have this column, return an array of all NULLs
+        let Some(column_index) = maybe_index else {
+            let num_row_groups = row_group_meta_data.count();
+            let sv = ScalarValue::try_from(self.field.data_type())?;
+            return sv.to_array_of_size(num_row_groups);
+        };
+
+        let stats_iter = row_group_meta_data.map(move |row_group_meta_data| {
+            row_group_meta_data.column(column_index).statistics()
+        });
+
+        // this is the value to use when the statistics are not set
+        let null_value = ScalarValue::try_from(self.field.data_type())?;
+        match mm {
+            MinMax::Min => {
+                let values = stats_iter.map(|column_statistics| {
+                    column_statistics
+                        .and_then(|column_statistics| {
+                            get_statistic!(
+                                column_statistics,
+                                min,
+                                min_bytes,
+                                Some(self.field.data_type().clone())
+                            )
+                        })
+                        .unwrap_or_else(|| null_value.clone())
+                });
+                ScalarValue::iter_to_array(values)
+            }
+            MinMax::Max => {
+                let values = stats_iter.map(|column_statistics| {
+                    column_statistics
+                        .and_then(|column_statistics| {
+                            get_statistic!(
+                                column_statistics,
+                                max,
+                                max_bytes,
+                                Some(self.field.data_type().clone())
+                            )
+                        })
+                        .unwrap_or_else(|| null_value.clone())
+                });
+                ScalarValue::iter_to_array(values)

Review Comment:
   It seems a shame to not just build the arrays from the outset, although perhaps this a temporary state



##########
datafusion/core/src/datasource/physical_plan/parquet/statistics.rs:
##########
@@ -0,0 +1,807 @@
+// 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.
+
+//! [`RowGroupStatisticsConverter`] tp converts parquet RowGroup statistics to arrow [`ArrayRef`].
+
+use arrow::{array::ArrayRef, datatypes::DataType};
+use arrow_array::new_empty_array;
+use arrow_schema::Field;
+use datafusion_common::{Result, ScalarValue};
+use parquet::file::{
+    metadata::RowGroupMetaData, statistics::Statistics as ParquetStatistics,
+};
+
+// 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(b))
+}
+
+// Copy from arrow-rs
+// https://github.com/apache/arrow-rs/blob/733b7e7fd1e8c43a404c3ce40ecf741d493c21b4/parquet/src/arrow/buffer/bit_util.rs#L55
+// Convert the byte slice to fixed length byte array with the length of 16
+fn sign_extend_be(b: &[u8]) -> [u8; 16] {
+    assert!(b.len() <= 16, "Array too large, expected less than 16");
+    let is_negative = (b[0] & 128u8) == 128u8;
+    let mut result = if is_negative { [255u8; 16] } else { [0u8; 16] };
+    for (d, s) in result.iter_mut().skip(16 - b.len()).zip(b) {
+        *d = *s;
+    }
+    result
+}
+
+/// Converts parquet RowGroup statistics (stored in
+/// [`RowGroupMetaData`]) into an arrow [`ArrayRef`]
+///
+/// For example, given a parquet file with 3 Row Groups, when asked for
+/// statistics for column "A" it will return a single array with 3 elements,
+///
+pub(crate) struct RowGroupStatisticsConverter<'a> {
+    field: &'a Field,
+}
+
+/// Extract a single min/max statistics from a [`ParquetStatistics`] object
+///
+/// * `$column_statistics` is the `ParquetStatistics` object
+/// * `$func is the function` (`min`/`max`) to call to get the value
+/// * `$bytes_func` is the function (`min_bytes`/`max_bytes`) to call to get the value as bytes
+/// * `$target_arrow_type` is the [`DataType`] of the target statistics
+macro_rules! get_statistic {
+    ($column_statistics:expr, $func:ident, $bytes_func:ident, $target_arrow_type:expr) => {{
+        if !$column_statistics.has_min_max_set() {
+            return None;
+        }
+        match $column_statistics {
+            ParquetStatistics::Boolean(s) => Some(ScalarValue::Boolean(Some(*s.$func()))),
+            ParquetStatistics::Int32(s) => {
+                match $target_arrow_type {
+                    // int32 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int32(Some(*s.$func()))),
+                }
+            }
+            ParquetStatistics::Int64(s) => {
+                match $target_arrow_type {
+                    // int64 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int64(Some(*s.$func()))),
+                }
+            }
+            // 96 bit ints not supported
+            ParquetStatistics::Int96(_) => None,
+            ParquetStatistics::Float(s) => Some(ScalarValue::Float32(Some(*s.$func()))),
+            ParquetStatistics::Double(s) => Some(ScalarValue::Float64(Some(*s.$func()))),
+            ParquetStatistics::ByteArray(s) => {
+                match $target_arrow_type {
+                    // decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => {
+                        let s = std::str::from_utf8(s.$bytes_func())
+                            .map(|s| s.to_string())
+                            .ok();
+                        Some(ScalarValue::Utf8(s))
+                    }
+                }
+            }
+            // type not supported yet
+            ParquetStatistics::FixedLenByteArray(s) => {
+                match $target_arrow_type {
+                    // just support the decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => None,
+                }
+            }
+        }
+    }};
+}
+
+#[derive(Debug, Clone, Copy)]
+enum MinMax {
+    Min,
+    Max,
+}
+
+impl<'a> RowGroupStatisticsConverter<'a> {
+    /// Create a new RowGoupStatisticsConverter suitable that can extract
+    /// statistics for the specified field
+    pub fn new(field: &'a Field) -> Self {
+        Self { field }
+    }
+
+    /// Returns the min value for the column into an array ref.
+    pub fn min<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(MinMax::Min, row_group_meta_data)
+    }
+
+    /// Returns the max value for the column into an array ref.
+    pub fn max<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(MinMax::Max, row_group_meta_data)
+    }
+
+    /// Extracts all min/max values for the column into an array ref.
+    fn min_max_impl<'b>(
+        &self,
+        mm: MinMax,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        let mut row_group_meta_data = row_group_meta_data.into_iter().peekable();
+
+        // if it is empty, return empty array
+        if row_group_meta_data.peek().is_none() {
+            return Ok(new_empty_array(self.field.data_type()));
+        }
+
+        let maybe_index = row_group_meta_data.peek().and_then(|rg_meta| {
+            rg_meta
+                .columns()
+                .iter()
+                .enumerate()
+                .find(|(_idx, c)| c.column_descr().name() == self.field.name())

Review Comment:
   Aside from being slow, this will be incorrect in the presence of nested fields



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Extract parquet statistics to its own module, add tests [arrow-datafusion]

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on code in PR #8294:
URL: https://github.com/apache/arrow-datafusion/pull/8294#discussion_r1402772669


##########
datafusion/core/src/datasource/physical_plan/parquet/statistics.rs:
##########
@@ -0,0 +1,807 @@
+// 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.
+
+//! [`RowGroupStatisticsConverter`] tp converts parquet RowGroup statistics to arrow [`ArrayRef`].
+
+use arrow::{array::ArrayRef, datatypes::DataType};
+use arrow_array::new_empty_array;
+use arrow_schema::Field;
+use datafusion_common::{Result, ScalarValue};
+use parquet::file::{
+    metadata::RowGroupMetaData, statistics::Statistics as ParquetStatistics,
+};
+
+// 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(b))
+}
+
+// Copy from arrow-rs
+// https://github.com/apache/arrow-rs/blob/733b7e7fd1e8c43a404c3ce40ecf741d493c21b4/parquet/src/arrow/buffer/bit_util.rs#L55
+// Convert the byte slice to fixed length byte array with the length of 16
+fn sign_extend_be(b: &[u8]) -> [u8; 16] {
+    assert!(b.len() <= 16, "Array too large, expected less than 16");
+    let is_negative = (b[0] & 128u8) == 128u8;
+    let mut result = if is_negative { [255u8; 16] } else { [0u8; 16] };
+    for (d, s) in result.iter_mut().skip(16 - b.len()).zip(b) {
+        *d = *s;
+    }
+    result
+}
+
+/// Converts parquet RowGroup statistics (stored in
+/// [`RowGroupMetaData`]) into an arrow [`ArrayRef`]
+///
+/// For example, given a parquet file with 3 Row Groups, when asked for
+/// statistics for column "A" it will return a single array with 3 elements,
+///
+pub(crate) struct RowGroupStatisticsConverter<'a> {
+    field: &'a Field,
+}
+
+/// Extract a single min/max statistics from a [`ParquetStatistics`] object
+///
+/// * `$column_statistics` is the `ParquetStatistics` object
+/// * `$func is the function` (`min`/`max`) to call to get the value
+/// * `$bytes_func` is the function (`min_bytes`/`max_bytes`) to call to get the value as bytes
+/// * `$target_arrow_type` is the [`DataType`] of the target statistics
+macro_rules! get_statistic {
+    ($column_statistics:expr, $func:ident, $bytes_func:ident, $target_arrow_type:expr) => {{
+        if !$column_statistics.has_min_max_set() {
+            return None;
+        }
+        match $column_statistics {
+            ParquetStatistics::Boolean(s) => Some(ScalarValue::Boolean(Some(*s.$func()))),
+            ParquetStatistics::Int32(s) => {
+                match $target_arrow_type {
+                    // int32 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int32(Some(*s.$func()))),
+                }
+            }
+            ParquetStatistics::Int64(s) => {
+                match $target_arrow_type {
+                    // int64 to decimal with the precision and scale
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(*s.$func() as i128),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => Some(ScalarValue::Int64(Some(*s.$func()))),
+                }
+            }
+            // 96 bit ints not supported
+            ParquetStatistics::Int96(_) => None,
+            ParquetStatistics::Float(s) => Some(ScalarValue::Float32(Some(*s.$func()))),
+            ParquetStatistics::Double(s) => Some(ScalarValue::Float64(Some(*s.$func()))),
+            ParquetStatistics::ByteArray(s) => {
+                match $target_arrow_type {
+                    // decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => {
+                        let s = std::str::from_utf8(s.$bytes_func())
+                            .map(|s| s.to_string())
+                            .ok();
+                        Some(ScalarValue::Utf8(s))
+                    }
+                }
+            }
+            // type not supported yet
+            ParquetStatistics::FixedLenByteArray(s) => {
+                match $target_arrow_type {
+                    // just support the decimal data type
+                    Some(DataType::Decimal128(precision, scale)) => {
+                        Some(ScalarValue::Decimal128(
+                            Some(from_bytes_to_i128(s.$bytes_func())),
+                            precision,
+                            scale,
+                        ))
+                    }
+                    _ => None,
+                }
+            }
+        }
+    }};
+}
+
+#[derive(Debug, Clone, Copy)]
+enum MinMax {
+    Min,
+    Max,
+}
+
+impl<'a> RowGroupStatisticsConverter<'a> {
+    /// Create a new RowGoupStatisticsConverter suitable that can extract
+    /// statistics for the specified field
+    pub fn new(field: &'a Field) -> Self {
+        Self { field }
+    }
+
+    /// Returns the min value for the column into an array ref.
+    pub fn min<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(MinMax::Min, row_group_meta_data)
+    }
+
+    /// Returns the max value for the column into an array ref.
+    pub fn max<'b>(
+        &self,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        self.min_max_impl(MinMax::Max, row_group_meta_data)
+    }
+
+    /// Extracts all min/max values for the column into an array ref.
+    fn min_max_impl<'b>(
+        &self,
+        mm: MinMax,
+        row_group_meta_data: impl IntoIterator<Item = &'b RowGroupMetaData>,
+    ) -> Result<ArrayRef> {
+        let mut row_group_meta_data = row_group_meta_data.into_iter().peekable();
+
+        // if it is empty, return empty array
+        if row_group_meta_data.peek().is_none() {
+            return Ok(new_empty_array(self.field.data_type()));
+        }
+
+        let maybe_index = row_group_meta_data.peek().and_then(|rg_meta| {
+            rg_meta
+                .columns()
+                .iter()
+                .enumerate()
+                .find(|(_idx, c)| c.column_descr().name() == self.field.name())
+                .map(|(idx, _c)| idx)
+        });
+
+        // don't have this column, return an array of all NULLs
+        let Some(column_index) = maybe_index else {
+            let num_row_groups = row_group_meta_data.count();
+            let sv = ScalarValue::try_from(self.field.data_type())?;
+            return sv.to_array_of_size(num_row_groups);
+        };
+
+        let stats_iter = row_group_meta_data.map(move |row_group_meta_data| {
+            row_group_meta_data.column(column_index).statistics()
+        });
+
+        // this is the value to use when the statistics are not set
+        let null_value = ScalarValue::try_from(self.field.data_type())?;
+        match mm {
+            MinMax::Min => {
+                let values = stats_iter.map(|column_statistics| {
+                    column_statistics
+                        .and_then(|column_statistics| {
+                            get_statistic!(
+                                column_statistics,
+                                min,
+                                min_bytes,
+                                Some(self.field.data_type().clone())
+                            )
+                        })
+                        .unwrap_or_else(|| null_value.clone())
+                });
+                ScalarValue::iter_to_array(values)
+            }
+            MinMax::Max => {
+                let values = stats_iter.map(|column_statistics| {
+                    column_statistics
+                        .and_then(|column_statistics| {
+                            get_statistic!(
+                                column_statistics,
+                                max,
+                                max_bytes,
+                                Some(self.field.data_type().clone())
+                            )
+                        })
+                        .unwrap_or_else(|| null_value.clone())
+                });
+                ScalarValue::iter_to_array(values)
+            }
+        }
+    }
+}
+
+#[cfg(test)]
+mod test {
+    use super::*;
+    use arrow_array::{
+        BinaryArray, BooleanArray, Decimal128Array, Float32Array, Float64Array,
+        Int32Array, Int64Array, RecordBatch, StringArray, TimestampNanosecondArray,
+    };
+    use arrow_schema::SchemaRef;
+    use bytes::Bytes;
+    use datafusion_common::test_util::parquet_test_data;
+    use parquet::arrow::arrow_reader::ArrowReaderBuilder;
+    use parquet::arrow::arrow_writer::ArrowWriter;
+    use parquet::file::metadata::ParquetMetaData;
+    use parquet::file::properties::{EnabledStatistics, WriterProperties};
+    use std::path::PathBuf;
+    use std::sync::Arc;
+
+    // TODO error cases (with parquet statistics that are mismatched in expected type)
+

Review Comment:
   Whilst I'm not entirely sure how repeated fields could be handled, I think a test of nested groups, i.e. StructArray, would be worthwhile to demonstrate the mismatch between arrow and parquet schema representations



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org