You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by "maxdebayser (via GitHub)" <gi...@apache.org> on 2023/06/13 18:10:01 UTC

[GitHub] [iceberg] maxdebayser opened a new pull request, #7831: Compute parquet stats

maxdebayser opened a new pull request, #7831:
URL: https://github.com/apache/iceberg/pull/7831

   @Fokko 
   
   This commit partly addresses issue https://github.com/apache/iceberg/issues/7256. Unfortunately the pyarrow library is not as flexible as we would like. When passing write_statistics=True to `pyarrow.parquet.write_table` the statistics are written out for each row group in the file, instead of computed globally.
   
   In the issue a "metadata_collector" was mentioned which I assume is the parameter of the `pyarrow.parquet.write_metadata` function. The `pyarrow.parquet.write_table` function has no such parameter.
   
   The function in this PR intentionally works at the level of individual parquet files instead of the dataset to support scenarios such as writing from Ray where each file of the dataset is written by a different task.


-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on pull request #7831: Compute parquet stats

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#issuecomment-1589945848

   Hey @maxdebayser thanks for raising this PR. I had something different in mind with the issue. The main problem here is that we pull a lot of data through Python, which brings in a lot of issues (the binary conversion that's probably expensive, and limited scalability due to the GIL).
   
   I just did a quick stab (and should have done that sooner), and found the following:
   
   ```python
   ➜  Desktop python3 
   Python 3.11.3 (main, Apr  7 2023, 20:13:31) [Clang 14.0.0 (clang-1400.0.29.202)] on darwin
   Type "help", "copyright", "credits" or "license" for more information.
   >>> import pyarrow as pa
   >>> table = pa.table({'n_legs': [2, 2, 4, 4, 5, 100],
   ...                   'animal': ["Flamingo", "Parrot", "Dog", "Horse",
   ...                              "Brittle stars", "Centipede"]})
   >>> metadata_collector = []
   >>> import pyarrow.parquet as pq
   >>> pq.write_to_dataset(
   ...     table, '/tmp/table',
   ...      metadata_collector=metadata_collector)
   >>> metadata_collector
   [<pyarrow._parquet.FileMetaData object at 0x11f955850>
     created_by: parquet-cpp-arrow version 11.0.0
     num_columns: 2
     num_rows: 6
     num_row_groups: 1
     format_version: 1.0
     serialized_size: 0]
   
   >>> metadata_collector[0].row_group(0)
   <pyarrow._parquet.RowGroupMetaData object at 0x105837d80>
     num_columns: 2
     num_rows: 6
     total_byte_size: 256
   
   >>> metadata_collector[0].row_group(0).to_dict()
   {
   	'num_columns': 2,
   	'num_rows': 6,
   	'total_byte_size': 256,
   	'columns': [{
   		'file_offset': 119,
   		'file_path': 'c569c5eaf90c4395885f31e012068b69-0.parquet',
   		'physical_type': 'INT64',
   		'num_values': 6,
   		'path_in_schema': 'n_legs',
   		'is_stats_set': True,
   		'statistics': {
   			'has_min_max': True,
   			'min': 2,
   			'max': 100,
   			'null_count': 0,
   			'distinct_count': 0,
   			'num_values': 6,
   			'physical_type': 'INT64'
   		},
   		'compression': 'SNAPPY',
   		'encodings': ('PLAIN_DICTIONARY', 'PLAIN', 'RLE'),
   		'has_dictionary_page': True,
   		'dictionary_page_offset': 4,
   		'data_page_offset': 46,
   		'total_compressed_size': 115,
   		'total_uncompressed_size': 117
   	}, {
   		'file_offset': 359,
   		'file_path': 'c569c5eaf90c4395885f31e012068b69-0.parquet',
   		'physical_type': 'BYTE_ARRAY',
   		'num_values': 6,
   		'path_in_schema': 'animal',
   		'is_stats_set': True,
   		'statistics': {
   			'has_min_max': True,
   			'min': 'Brittle stars',
   			'max': 'Parrot',
   			'null_count': 0,
   			'distinct_count': 0,
   			'num_values': 6,
   			'physical_type': 'BYTE_ARRAY'
   		},
   		'compression': 'SNAPPY',
   		'encodings': ('PLAIN_DICTIONARY', 'PLAIN', 'RLE'),
   		'has_dictionary_page': True,
   		'dictionary_page_offset': 215,
   		'data_page_offset': 302,
   		'total_compressed_size': 144,
   		'total_uncompressed_size': 139
   	}]
   }
   ```
   
   I think it is much better to retrieve the min-max from there. This is done by PyArrow and is probably much faster than when we do it in Python. I really would like to stay away from processing data as much as possible.
   
   I think the complexity here is to map it back to the original columns (and it gets tricky I think with nested columns).


-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] maxdebayser commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "maxdebayser (via GitHub)" <gi...@apache.org>.
maxdebayser commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1258819386


##########
python/pyiceberg/utils/file_stats.py:
##########
@@ -0,0 +1,333 @@
+#  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.
+
+import struct
+from typing import (
+    Any,
+    Dict,
+    List,
+    Union,
+)
+
+import pyarrow.lib
+import pyarrow.parquet as pq
+
+from pyiceberg.manifest import DataFile, FileFormat
+from pyiceberg.schema import Schema, SchemaVisitor, visit
+from pyiceberg.types import (
+    IcebergType,
+    ListType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StructType,
+)
+
+BOUND_TRUNCATED_LENGHT = 16
+
+# Serialization rules: https://iceberg.apache.org/spec/#binary-single-value-serialization
+#
+# Type      Binary serialization
+# boolean   0x00 for false, non-zero byte for true
+# int       Stored as 4-byte little-endian
+# long      Stored as 8-byte little-endian
+# float     Stored as 4-byte little-endian
+# double    Stored as 8-byte little-endian
+# date      Stores days from the 1970-01-01 in an 4-byte little-endian int
+# time      Stores microseconds from midnight in an 8-byte little-endian long
+# timestamp without zone	Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long
+# timestamp with zone	Stores microseconds from 1970-01-01 00:00:00.000000 UTC in an 8-byte little-endian long
+# string    UTF-8 bytes (without length)
+# uuid      16-byte big-endian value, see example in Appendix B
+# fixed(L)  Binary value
+# binary    Binary value (without length)
+#
+
+
+def bool_to_avro(value: bool) -> bytes:
+    return struct.pack("?", value)

Review Comment:
   Sure



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1264727823


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1013,3 +1025,271 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+class StatsAggregator:
+    def __init__(self, type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.trunc_length = trunc_length
+        self.primitive_type: Optional[PrimitiveType] = None
+
+        if type_string == "BOOLEAN":
+            self.primitive_type = BooleanType()
+        elif type_string == "INT32":
+            self.primitive_type = IntegerType()
+        elif type_string == "INT64":
+            self.primitive_type = LongType()
+        elif type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+        elif type_string == "FLOAT":
+            self.primitive_type = FloatType()
+        elif type_string == "DOUBLE":
+            self.primitive_type = DoubleType()
+        elif type_string == "BYTE_ARRAY":
+            self.primitive_type = BinaryType()
+        elif type_string == "FIXED_LEN_BYTE_ARRAY":
+            self.primitive_type = BinaryType()
+        else:
+            raise AssertionError(f"Unknown physical type {type_string}")
+
+    def serialize(self, value: Any) -> bytes:
+        if type(value) == str:
+            value = value.encode()
+        assert self.primitive_type is not None  # appease mypy
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        if self.current_min is None:
+            self.current_min = val
+        else:
+            self.current_min = min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        if self.current_max is None:
+            self.current_max = val
+        else:
+            self.current_max = max(self.current_max, val)
+
+    def get_min(self) -> bytes:
+        return self.serialize(self.current_min)[: self.trunc_length]
+
+    def get_max(self) -> bytes:
+        return self.serialize(self.current_max)[: self.trunc_length]
+
+
+DEFAULT_TRUNCATION_LENGHT = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+
+class MetricModeTypes(Enum):
+    TRUNCATE = "truncate"
+    NONE = "none"
+    COUNTS = "counts"
+    FULL = "full"
+
+
+DEFAULT_METRICS_MODE_KEY = "write.metadata.metrics.default"
+COLUMN_METRICS_MODE_KEY = "write.metadata.metrics.column"
+
+
+@dataclass(frozen=True)
+class MetricsMode(Singleton):
+    type: MetricModeTypes
+    length: Optional[int] = None
+
+
+def match_metrics_mode(mode: str) -> MetricsMode:
+    m = re.match(TRUNCATION_EXPR, mode, re.IGNORECASE)
+    if m:
+        length = int(m[1])
+        if length < 1:
+            raise AssertionError("Truncation length must be larger than 0")
+        return MetricsMode(MetricModeTypes.TRUNCATE, int(m[1]))
+    elif re.match("^none$", mode, re.IGNORECASE):
+        return MetricsMode(MetricModeTypes.NONE)
+    elif re.match("^counts$", mode, re.IGNORECASE):
+        return MetricsMode(MetricModeTypes.COUNTS)
+    elif re.match("^full$", mode, re.IGNORECASE):
+        return MetricsMode(MetricModeTypes.FULL)
+    else:
+        raise AssertionError(f"Unsupported metrics mode {mode}")
+
+
+@dataclass(frozen=True)
+class StatisticsCollector:
+    field_id: int
+    iceberg_type: PrimitiveType
+    mode: MetricsMode
+    column_name: str
+
+
+class PyArrowStatisticsCollector(PreOrderSchemaVisitor[List[StatisticsCollector]]):
+    _field_id = 0
+    _schema: Schema
+    _properties: Dict[str, str]
+
+    def __init__(self, schema: Schema, properties: Dict[str, str]):
+        self._schema = schema
+        self._properties = properties
+
+    def schema(self, schema: Schema, struct_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        return struct_result()
+
+    def struct(
+        self, struct: StructType, field_results: List[Callable[[], List[StatisticsCollector]]]
+    ) -> List[StatisticsCollector]:
+        return list(chain(*[result() for result in field_results]))
+
+    def field(self, field: NestedField, field_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = field.field_id
+        result = field_result()
+        return result
+
+    def list(self, list_type: ListType, element_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = list_type.element_id
+        return element_result()
+
+    def map(
+        self,
+        map_type: MapType,
+        key_result: Callable[[], List[StatisticsCollector]],
+        value_result: Callable[[], List[StatisticsCollector]],
+    ) -> List[StatisticsCollector]:
+        self._field_id = map_type.key_id
+        k = key_result()
+        self._field_id = map_type.value_id
+        v = value_result()
+        return k + v
+
+    def primitive(self, primitive: PrimitiveType) -> List[StatisticsCollector]:
+        column_name = self._schema.find_column_name(self._field_id)
+        assert column_name is not None, f"Column for field {self._field_id} not found"
+
+        metrics_mode = MetricsMode(MetricModeTypes.TRUNCATE, DEFAULT_TRUNCATION_LENGHT)
+
+        default_mode = self._properties.get(DEFAULT_METRICS_MODE_KEY)
+        if default_mode:
+            metrics_mode = match_metrics_mode(default_mode)
+
+        col_mode = self._properties.get(f"{COLUMN_METRICS_MODE_KEY}.{column_name}")
+        if col_mode:
+            metrics_mode = match_metrics_mode(col_mode)
+
+        return [StatisticsCollector(field_id=self._field_id, iceberg_type=primitive, mode=metrics_mode, column_name=column_name)]
+
+
+def fill_parquet_file_metadata(
+    df: DataFile,
+    parquet_metadata: pq.FileMetaData,
+    file_size: int,
+    table_metadata: TableMetadata,
+) -> None:
+    """
+    Computes and fills the following fields of the DataFile object.
+
+    - file_format
+    - record_count
+    - file_size_in_bytes
+    - column_sizes
+    - value_counts
+    - null_value_counts
+    - nan_value_counts
+    - lower_bounds
+    - upper_bounds
+    - split_offsets
+
+    Args:
+        df (DataFile): A DataFile object representing the Parquet file for which metadata is to be filled.
+        parquet_metadata (pyarrow.parquet.FileMetaData): A pyarrow metadata object.
+        file_size (int): The total compressed file size cannot be retrieved from the metadata and hence has to
+            be passed here. Depending on the kind of file system and pyarrow library call used, different
+            ways to obtain this value might be appropriate.
+        table_metadata (pyiceberg.table.metadata.TableMetadata): The Iceberg table metadata. It is required to
+            compute the mapping if column position to iceberg schema type id. It's also used to set the mode
+            for column metrics collection
+    """
+    schema = next(filter(lambda s: s.schema_id == table_metadata.current_schema_id, table_metadata.schemas))
+
+    stats_columns = pre_order_visit(schema, PyArrowStatisticsCollector(schema, table_metadata.properties))
+    assert parquet_metadata.num_columns == len(
+        stats_columns
+    ), f"Number of columns in metadata ({len(stats_columns)}) is different from the number of columns in pyarrow table ({parquet_metadata.num_columns})"
+
+    column_sizes: Dict[int, int] = {}
+    value_counts: Dict[int, int] = {}
+    split_offsets: List[int] = []
+
+    null_value_counts: Dict[int, int] = {}
+    nan_value_counts: Dict[int, int] = {}
+
+    col_aggs = {}
+
+    for r in range(parquet_metadata.num_row_groups):
+        # References:
+        # https://github.com/apache/iceberg/blob/fc381a81a1fdb8f51a0637ca27cd30673bd7aad3/parquet/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java#L232
+        # https://github.com/apache/parquet-mr/blob/ac29db4611f86a07cc6877b416aa4b183e09b353/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/ColumnChunkMetaData.java#L184
+
+        row_group = parquet_metadata.row_group(r)
+
+        data_offset = row_group.column(0).data_page_offset
+        dictionary_offset = row_group.column(0).dictionary_page_offset
+
+        if row_group.column(0).has_dictionary_page and dictionary_offset < data_offset:
+            split_offsets.append(dictionary_offset)
+        else:
+            split_offsets.append(data_offset)
+
+        for c in range(parquet_metadata.num_columns):
+            col_id = stats_columns[c].field_id
+
+            column = row_group.column(c)
+
+            column_sizes[col_id] = column_sizes.get(col_id, 0) + column.total_compressed_size
+
+            metrics_mode = stats_columns[c].mode
+
+            if metrics_mode == MetricsMode(MetricModeTypes.NONE):
+                continue
+
+            value_counts[col_id] = value_counts.get(col_id, 0) + column.num_values
+
+            if column.is_stats_set:
+                try:
+                    statistics = column.statistics
+
+                    null_value_counts[col_id] = null_value_counts.get(col_id, 0) + statistics.null_count
+
+                    if metrics_mode == MetricsMode(MetricModeTypes.COUNTS):
+                        continue
+
+                    if col_id not in col_aggs:
+                        col_aggs[col_id] = StatsAggregator(statistics.physical_type, metrics_mode.length)
+
+                    col_aggs[col_id].add_min(statistics.min)
+                    col_aggs[col_id].add_max(statistics.max)

Review Comment:
   What do you think of the following? This way we have fewer lookups. Also, I think it is best to stick with the Iceberg naming (field-id over column-id).
   ```suggestion
           for pos, stats_col in enumerate(stats_columns):
               field_id = stats_col.field_id
   
               column = row_group.column(pos)
   
               column_sizes[field_id] = column_sizes.get(field_id, 0) + column.total_compressed_size
   
               if stats_col.mode == MetricsMode(MetricModeTypes.NONE):
                   continue
   
               value_counts[field_id] = value_counts.get(field_id, 0) + column.num_values
   
               if column.is_stats_set:
                   try:
                       statistics = column.statistics
   
                       null_value_counts[field_id] = null_value_counts.get(field_id, 0) + statistics.null_count
   
                       if stats_col.mode == MetricsMode(MetricModeTypes.COUNTS):
                           continue
   
                       if field_id not in col_aggs:
                           col_aggs[field_id] = StatsAggregator(stats_col.iceberg_type, stats_col.mode.length)
   
                       col_aggs[field_id].add_min(statistics.min)
                       col_aggs[field_id].add_max(statistics.max)
   ```
   Also, this way we already know the primitive type., and we don't have to look it up.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1285259118


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1039,335 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.values()).union({"INT96"})
+
+
+class StatsAggregator:
+    current_min: Any
+    current_max: Any
+    trunc_length: Optional[int]
+
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min = None
+        self.current_max = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        expected_physical_type = _PRIMITIVE_TO_PHYSICAL[iceberg_type]
+        if expected_physical_type != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {expected_physical_type}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        self.current_min = val if self.current_min is None else min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        self.current_max = val if self.current_max is None else max(val, self.current_max)
+
+    def get_min(self) -> bytes:
+        return self.serialize(
+            self.current_min
+            if self.trunc_length is None
+            else TruncateTransform(width=self.trunc_length).transform(self.primitive_type)(self.current_min)
+        )
+
+    def get_max(self) -> Optional[bytes]:
+        if self.current_max is None:
+            return None
+
+        if self.primitive_type == StringType():
+            if type(self.current_max) != str:
+                raise ValueError("Expected the current_max to be a string")
+
+            s_result = self.current_max[: self.trunc_length]
+            if s_result != self.current_max:
+                chars = [*s_result]
+
+                for i in range(-1, -len(s_result) - 1, -1):
+                    try:
+                        to_inc = ord(chars[i])
+                        # will raise exception if the highest unicode code is reached
+                        _next = chr(to_inc + 1)
+                        chars[i] = _next
+                        return self.serialize("".join(chars))
+                    except ValueError:
+                        pass
+                return None  # didn't find a valid upper bound
+            return self.serialize(s_result)
+        elif self.primitive_type == BinaryType():
+            if type(self.current_max) != bytes:
+                raise ValueError("Expected the current_max to be bytes")
+            b_result = self.current_max[: self.trunc_length]
+            if b_result != self.current_max:
+                _bytes = [*b_result]
+                for i in range(-1, -len(b_result) - 1, -1):
+                    if _bytes[i] < 255:
+                        _bytes[i] += 1
+                        return b"".join([i.to_bytes(1, byteorder="little") for i in _bytes])
+                return None
+
+            return self.serialize(b_result)
+        else:
+            return self.serialize(self.current_max)[: self.trunc_length]
+
+
+DEFAULT_TRUNCATION_LENGTH = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+
+class MetricModeTypes(Enum):
+    TRUNCATE = "truncate"
+    NONE = "none"
+    COUNTS = "counts"
+    FULL = "full"
+
+
+DEFAULT_METRICS_MODE_KEY = "write.metadata.metrics.default"
+COLUMN_METRICS_MODE_KEY_PREFIX = "write.metadata.metrics.column"
+
+
+@dataclass(frozen=True)
+class MetricsMode(Singleton):
+    type: MetricModeTypes
+    length: Optional[int] = None
+
+
+def match_metrics_mode(mode: str) -> MetricsMode:
+    sanitized_mode = mode.lower()
+    if sanitized_mode.startswith("truncate"):
+        m = re.match(TRUNCATION_EXPR, mode, re.IGNORECASE)
+        if m:
+            length = int(m[1])
+            if length < 1:
+                raise ValueError("Truncation length must be larger than 0")
+            return MetricsMode(MetricModeTypes.TRUNCATE, int(m[1]))
+        else:
+            raise ValueError(f"Malformed truncate: {mode}")
+    elif sanitized_mode.startswith("none"):
+        return MetricsMode(MetricModeTypes.NONE)
+    elif sanitized_mode.startswith("counts"):
+        return MetricsMode(MetricModeTypes.COUNTS)
+    elif sanitized_mode.startswith("full"):
+        return MetricsMode(MetricModeTypes.FULL)
+    else:
+        raise ValueError(f"Unsupported metrics mode: {mode}")
+
+
+@dataclass(frozen=True)
+class StatisticsCollector:
+    field_id: int
+    iceberg_type: PrimitiveType
+    mode: MetricsMode
+    column_name: str
+
+
+class PyArrowStatisticsCollector(PreOrderSchemaVisitor[List[StatisticsCollector]]):
+    _field_id: int = 0
+    _schema: Schema
+    _properties: Dict[str, str]
+
+    def __init__(self, schema: Schema, properties: Dict[str, str]):
+        self._schema = schema
+        self._properties = properties
+
+    def schema(self, schema: Schema, struct_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        return struct_result()
+
+    def struct(
+        self, struct: StructType, field_results: List[Callable[[], List[StatisticsCollector]]]
+    ) -> List[StatisticsCollector]:
+        return list(chain(*[result() for result in field_results]))
+
+    def field(self, field: NestedField, field_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = field.field_id
+        result = field_result()
+        return result
+
+    def list(self, list_type: ListType, element_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = list_type.element_id
+        return element_result()
+
+    def map(
+        self,
+        map_type: MapType,
+        key_result: Callable[[], List[StatisticsCollector]],
+        value_result: Callable[[], List[StatisticsCollector]],
+    ) -> List[StatisticsCollector]:
+        self._field_id = map_type.key_id
+        k = key_result()
+        self._field_id = map_type.value_id
+        v = value_result()
+        return k + v
+
+    def primitive(self, primitive: PrimitiveType) -> List[StatisticsCollector]:
+        column_name = self._schema.find_column_name(self._field_id)
+        if column_name is None:
+            raise ValueError(f"Column for field {self._field_id} not found")
+
+        metrics_mode = MetricsMode(MetricModeTypes.TRUNCATE, DEFAULT_TRUNCATION_LENGTH)

Review Comment:
   Should the lhs be a constant?



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1039,335 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.values()).union({"INT96"})
+
+
+class StatsAggregator:
+    current_min: Any
+    current_max: Any
+    trunc_length: Optional[int]
+
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min = None
+        self.current_max = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        expected_physical_type = _PRIMITIVE_TO_PHYSICAL[iceberg_type]
+        if expected_physical_type != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {expected_physical_type}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        self.current_min = val if self.current_min is None else min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        self.current_max = val if self.current_max is None else max(val, self.current_max)
+
+    def get_min(self) -> bytes:
+        return self.serialize(
+            self.current_min
+            if self.trunc_length is None
+            else TruncateTransform(width=self.trunc_length).transform(self.primitive_type)(self.current_min)
+        )
+
+    def get_max(self) -> Optional[bytes]:
+        if self.current_max is None:
+            return None
+
+        if self.primitive_type == StringType():
+            if type(self.current_max) != str:
+                raise ValueError("Expected the current_max to be a string")
+
+            s_result = self.current_max[: self.trunc_length]
+            if s_result != self.current_max:
+                chars = [*s_result]
+
+                for i in range(-1, -len(s_result) - 1, -1):
+                    try:
+                        to_inc = ord(chars[i])
+                        # will raise exception if the highest unicode code is reached
+                        _next = chr(to_inc + 1)
+                        chars[i] = _next
+                        return self.serialize("".join(chars))
+                    except ValueError:
+                        pass
+                return None  # didn't find a valid upper bound
+            return self.serialize(s_result)
+        elif self.primitive_type == BinaryType():
+            if type(self.current_max) != bytes:
+                raise ValueError("Expected the current_max to be bytes")
+            b_result = self.current_max[: self.trunc_length]
+            if b_result != self.current_max:
+                _bytes = [*b_result]
+                for i in range(-1, -len(b_result) - 1, -1):
+                    if _bytes[i] < 255:
+                        _bytes[i] += 1
+                        return b"".join([i.to_bytes(1, byteorder="little") for i in _bytes])
+                return None
+
+            return self.serialize(b_result)
+        else:
+            return self.serialize(self.current_max)[: self.trunc_length]
+
+
+DEFAULT_TRUNCATION_LENGTH = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+
+class MetricModeTypes(Enum):
+    TRUNCATE = "truncate"
+    NONE = "none"
+    COUNTS = "counts"
+    FULL = "full"
+
+
+DEFAULT_METRICS_MODE_KEY = "write.metadata.metrics.default"
+COLUMN_METRICS_MODE_KEY_PREFIX = "write.metadata.metrics.column"
+
+
+@dataclass(frozen=True)
+class MetricsMode(Singleton):
+    type: MetricModeTypes
+    length: Optional[int] = None
+
+
+def match_metrics_mode(mode: str) -> MetricsMode:
+    sanitized_mode = mode.lower()
+    if sanitized_mode.startswith("truncate"):
+        m = re.match(TRUNCATION_EXPR, mode, re.IGNORECASE)
+        if m:
+            length = int(m[1])
+            if length < 1:
+                raise ValueError("Truncation length must be larger than 0")
+            return MetricsMode(MetricModeTypes.TRUNCATE, int(m[1]))
+        else:
+            raise ValueError(f"Malformed truncate: {mode}")
+    elif sanitized_mode.startswith("none"):
+        return MetricsMode(MetricModeTypes.NONE)
+    elif sanitized_mode.startswith("counts"):
+        return MetricsMode(MetricModeTypes.COUNTS)
+    elif sanitized_mode.startswith("full"):
+        return MetricsMode(MetricModeTypes.FULL)
+    else:
+        raise ValueError(f"Unsupported metrics mode: {mode}")
+
+
+@dataclass(frozen=True)
+class StatisticsCollector:
+    field_id: int
+    iceberg_type: PrimitiveType
+    mode: MetricsMode
+    column_name: str
+
+
+class PyArrowStatisticsCollector(PreOrderSchemaVisitor[List[StatisticsCollector]]):
+    _field_id: int = 0
+    _schema: Schema
+    _properties: Dict[str, str]
+
+    def __init__(self, schema: Schema, properties: Dict[str, str]):
+        self._schema = schema
+        self._properties = properties
+
+    def schema(self, schema: Schema, struct_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        return struct_result()
+
+    def struct(
+        self, struct: StructType, field_results: List[Callable[[], List[StatisticsCollector]]]
+    ) -> List[StatisticsCollector]:
+        return list(chain(*[result() for result in field_results]))
+
+    def field(self, field: NestedField, field_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = field.field_id
+        result = field_result()
+        return result
+
+    def list(self, list_type: ListType, element_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = list_type.element_id
+        return element_result()
+
+    def map(
+        self,
+        map_type: MapType,
+        key_result: Callable[[], List[StatisticsCollector]],
+        value_result: Callable[[], List[StatisticsCollector]],
+    ) -> List[StatisticsCollector]:
+        self._field_id = map_type.key_id
+        k = key_result()
+        self._field_id = map_type.value_id
+        v = value_result()
+        return k + v
+
+    def primitive(self, primitive: PrimitiveType) -> List[StatisticsCollector]:
+        column_name = self._schema.find_column_name(self._field_id)
+        if column_name is None:
+            raise ValueError(f"Column for field {self._field_id} not found")
+
+        metrics_mode = MetricsMode(MetricModeTypes.TRUNCATE, DEFAULT_TRUNCATION_LENGTH)

Review Comment:
   Should the rhs be a constant?



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1285258091


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1039,335 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.values()).union({"INT96"})
+
+
+class StatsAggregator:
+    current_min: Any
+    current_max: Any
+    trunc_length: Optional[int]
+
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min = None
+        self.current_max = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        expected_physical_type = _PRIMITIVE_TO_PHYSICAL[iceberg_type]
+        if expected_physical_type != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {expected_physical_type}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        self.current_min = val if self.current_min is None else min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        self.current_max = val if self.current_max is None else max(val, self.current_max)
+
+    def get_min(self) -> bytes:
+        return self.serialize(
+            self.current_min
+            if self.trunc_length is None
+            else TruncateTransform(width=self.trunc_length).transform(self.primitive_type)(self.current_min)
+        )
+
+    def get_max(self) -> Optional[bytes]:
+        if self.current_max is None:
+            return None
+
+        if self.primitive_type == StringType():
+            if type(self.current_max) != str:
+                raise ValueError("Expected the current_max to be a string")
+
+            s_result = self.current_max[: self.trunc_length]

Review Comment:
   I don't think this is a good place to put this logic. I would prefer adding it in a separate PR with dedicated tests for the util method that produces a truncated upper bound.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1284501717


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1040,337 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = ["BOOLEAN", "INT32", "INT64", "INT96", "FLOAT", "DOUBLE", "BYTE_ARRAY", "FIXED_LEN_BYTE_ARRAY"]
+
+
+class StatsAggregator:

Review Comment:
   ```suggestion
   class StatsAggregator:
       current_min: Any
       current_max: Any
       trunc_length: Optional[int]
   
   ```



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1040,337 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = ["BOOLEAN", "INT32", "INT64", "INT96", "FLOAT", "DOUBLE", "BYTE_ARRAY", "FIXED_LEN_BYTE_ARRAY"]
+
+
+class StatsAggregator:
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        if _PRIMITIVE_TO_PHYSICAL[iceberg_type] != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {_PRIMITIVE_TO_PHYSICAL[iceberg_type]}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if type(value) == date:
+            value = date_to_days(value)
+        elif type(value) == time:
+            value = time_object_to_micros(value)
+        elif type(value) == datetime:
+            value = datetime_to_micros(value)
+
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        if self.current_min is None:
+            self.current_min = val
+        else:
+            self.current_min = min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        if self.current_max is None:
+            self.current_max = val
+        else:
+            self.current_max = max(self.current_max, val)

Review Comment:
   ```suggestion
           self.current_max = val if self.current_max is None else max(val, self.current_max)
   ```



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1040,337 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = ["BOOLEAN", "INT32", "INT64", "INT96", "FLOAT", "DOUBLE", "BYTE_ARRAY", "FIXED_LEN_BYTE_ARRAY"]
+
+
+class StatsAggregator:
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        if _PRIMITIVE_TO_PHYSICAL[iceberg_type] != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {_PRIMITIVE_TO_PHYSICAL[iceberg_type]}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if type(value) == date:
+            value = date_to_days(value)
+        elif type(value) == time:
+            value = time_object_to_micros(value)
+        elif type(value) == datetime:
+            value = datetime_to_micros(value)
+
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        if self.current_min is None:
+            self.current_min = val
+        else:
+            self.current_min = min(val, self.current_min)

Review Comment:
   ```suggestion
           self.current_min = val if self.current_min is None else min(val, self.current_min)
   ```



##########
python/tests/io/test_pyarrow.py:
##########
@@ -1345,3 +1373,659 @@ def test_pyarrow_wrap_fsspec(example_task: FileScanTask, table_schema_simple: Sc
 bar: [[1,2,3]]
 baz: [[true,false,null]]"""
     )
+
+
+def construct_test_table() -> pa.Buffer:
+    table_metadata = {
+        "format-version": 2,
+        "location": "s3://bucket/test/location",
+        "last-column-id": 7,
+        "current-schema-id": 0,
+        "schemas": [
+            {
+                "type": "struct",
+                "schema-id": 0,
+                "fields": [
+                    {"id": 1, "name": "strings", "required": False, "type": "string"},
+                    {"id": 2, "name": "floats", "required": False, "type": "float"},
+                    {
+                        "id": 3,
+                        "name": "list",
+                        "required": False,
+                        "type": {"type": "list", "element-id": 5, "element": "long", "element-required": False},
+                    },
+                    {
+                        "id": 4,
+                        "name": "maps",
+                        "required": False,
+                        "type": {
+                            "type": "map",
+                            "key-id": 6,
+                            "key": "long",
+                            "value-id": 7,
+                            "value": "long",
+                            "value-required": False,
+                        },
+                    },
+                ],
+            },
+        ],
+        "default-spec-id": 0,
+        "partition-specs": [{"spec-id": 0, "fields": []}],
+        "properties": {},
+    }
+
+    table_metadata = TableMetadataUtil.parse_obj(table_metadata)
+    arrow_schema = schema_to_pyarrow(table_metadata.schemas[0])
+
+    _strings = ["zzzzzzzzzzzzzzzzzzzz", "rrrrrrrrrrrrrrrrrrrr", None, "aaaaaaaaaaaaaaaaaaaa"]
+
+    _floats = [3.14, math.nan, 1.69, 100]
+
+    _list = [[1, 2, 3], [4, 5, 6], None, [7, 8, 9]]
+
+    _maps: List[Optional[Dict[int, int]]] = [
+        {1: 2, 3: 4},
+        None,
+        {5: 6},
+        {},
+    ]
+
+    table = pa.Table.from_pydict(
+        {
+            "strings": _strings,
+            "floats": _floats,
+            "list": _list,
+            "maps": _maps,
+        },
+        schema=arrow_schema,
+    )
+    f = pa.BufferOutputStream()
+
+    metadata_collector: List[Any] = []
+    writer = pq.ParquetWriter(f, table.schema, metadata_collector=metadata_collector)
+
+    writer.write_table(table)
+    writer.close()

Review Comment:
   ```suggestion
       metadata_collector: List[Any] = []
       with pa.BufferOutputStream() as f:
           with pq.ParquetWriter(f, table.schema, metadata_collector=metadata_collector) as writer:
               writer.write_table(table)
   ```



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1040,337 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = ["BOOLEAN", "INT32", "INT64", "INT96", "FLOAT", "DOUBLE", "BYTE_ARRAY", "FIXED_LEN_BYTE_ARRAY"]
+
+
+class StatsAggregator:
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        if _PRIMITIVE_TO_PHYSICAL[iceberg_type] != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {_PRIMITIVE_TO_PHYSICAL[iceberg_type]}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if type(value) == date:
+            value = date_to_days(value)
+        elif type(value) == time:
+            value = time_object_to_micros(value)
+        elif type(value) == datetime:
+            value = datetime_to_micros(value)
+
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        if self.current_min is None:
+            self.current_min = val
+        else:
+            self.current_min = min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        if self.current_max is None:
+            self.current_max = val
+        else:
+            self.current_max = max(self.current_max, val)
+
+    def get_min(self) -> bytes:
+        if self.primitive_type == StringType():
+            if type(self.current_min) != str:
+                raise ValueError("Expected the current_min to be a string")
+            return self.serialize(self.current_min[: self.trunc_length])
+        else:
+            return self.serialize(self.current_min)[: self.trunc_length]
+
+    def get_max(self) -> Optional[bytes]:
+        if self.primitive_type == StringType():
+            if type(self.current_max) != str:
+                raise ValueError("Expected the current_max to be a string")
+
+            s_result = self.current_max[: self.trunc_length]
+            if s_result != self.current_max:
+                chars = [*s_result]
+
+                for i in range(-1, -len(s_result) - 1, -1):
+                    try:
+                        to_inc = ord(chars[i])
+                        # will raise exception if the highest unicode code is reached
+                        _next = chr(to_inc + 1)
+                        chars[i] = _next
+                        return self.serialize("".join(chars))
+                    except ValueError:
+                        pass
+                return None  # didn't find a valid upper bound
+            return self.serialize(s_result)
+        elif self.primitive_type == BinaryType():
+            if type(self.current_max) != bytes:
+                raise ValueError("Expected the current_max to be bytes")
+            b_result = self.current_max[: self.trunc_length]
+            if b_result != self.current_max:
+                _bytes = [*b_result]
+                for i in range(-1, -len(b_result) - 1, -1):
+                    if _bytes[i] < 255:
+                        _bytes[i] += 1
+                        return b"".join([i.to_bytes(1, byteorder="little") for i in _bytes])
+                return None
+
+            return self.serialize(b_result)
+        else:
+            return self.serialize(self.current_max)[: self.trunc_length]
+
+
+DEFAULT_TRUNCATION_LENGHT = 16

Review Comment:
   ```suggestion
   DEFAULT_TRUNCATION_LENGTH = 16
   ```



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1040,337 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = ["BOOLEAN", "INT32", "INT64", "INT96", "FLOAT", "DOUBLE", "BYTE_ARRAY", "FIXED_LEN_BYTE_ARRAY"]
+
+
+class StatsAggregator:
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        if _PRIMITIVE_TO_PHYSICAL[iceberg_type] != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {_PRIMITIVE_TO_PHYSICAL[iceberg_type]}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if type(value) == date:
+            value = date_to_days(value)
+        elif type(value) == time:
+            value = time_object_to_micros(value)
+        elif type(value) == datetime:
+            value = datetime_to_micros(value)
+
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        if self.current_min is None:
+            self.current_min = val
+        else:
+            self.current_min = min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        if self.current_max is None:
+            self.current_max = val
+        else:
+            self.current_max = max(self.current_max, val)
+
+    def get_min(self) -> bytes:
+        if self.primitive_type == StringType():
+            if type(self.current_min) != str:
+                raise ValueError("Expected the current_min to be a string")
+            return self.serialize(self.current_min[: self.trunc_length])
+        else:
+            return self.serialize(self.current_min)[: self.trunc_length]

Review Comment:
   ```suggestion
           return self.serialize(
               self.current_min
               if self.trunc_length is None
               else TruncateTransform(width=self.trunc_length).transform(self.primitive_type)(self.current_min)
           )
   ```



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1040,291 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHISICAL_TYPES = ["BOOLEAN", "INT32", "INT64", "INT96", "FLOAT", "DOUBLE", "BYTE_ARRAY", "FIXED_LEN_BYTE_ARRAY"]
+
+
+class StatsAggregator:
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.trunc_length = trunc_length
+
+        assert physical_type_string in _PHISICAL_TYPES, f"Unknown physical type {physical_type_string}"
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+        assert (
+            _PRIMITIVE_TO_PHYSICAL[iceberg_type] == physical_type_string
+        ), f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {_PRIMITIVE_TO_PHYSICAL[iceberg_type]}"
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if type(value) == date:

Review Comment:
   The PR is in :)



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1040,337 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = ["BOOLEAN", "INT32", "INT64", "INT96", "FLOAT", "DOUBLE", "BYTE_ARRAY", "FIXED_LEN_BYTE_ARRAY"]
+
+
+class StatsAggregator:
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        if _PRIMITIVE_TO_PHYSICAL[iceberg_type] != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {_PRIMITIVE_TO_PHYSICAL[iceberg_type]}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if type(value) == date:
+            value = date_to_days(value)
+        elif type(value) == time:
+            value = time_object_to_micros(value)
+        elif type(value) == datetime:
+            value = datetime_to_micros(value)
+
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        if self.current_min is None:
+            self.current_min = val
+        else:
+            self.current_min = min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        if self.current_max is None:
+            self.current_max = val
+        else:
+            self.current_max = max(self.current_max, val)
+
+    def get_min(self) -> bytes:
+        if self.primitive_type == StringType():
+            if type(self.current_min) != str:
+                raise ValueError("Expected the current_min to be a string")
+            return self.serialize(self.current_min[: self.trunc_length])
+        else:
+            return self.serialize(self.current_min)[: self.trunc_length]
+
+    def get_max(self) -> Optional[bytes]:
+        if self.primitive_type == StringType():
+            if type(self.current_max) != str:
+                raise ValueError("Expected the current_max to be a string")
+
+            s_result = self.current_max[: self.trunc_length]
+            if s_result != self.current_max:
+                chars = [*s_result]
+
+                for i in range(-1, -len(s_result) - 1, -1):
+                    try:
+                        to_inc = ord(chars[i])
+                        # will raise exception if the highest unicode code is reached
+                        _next = chr(to_inc + 1)
+                        chars[i] = _next
+                        return self.serialize("".join(chars))
+                    except ValueError:
+                        pass
+                return None  # didn't find a valid upper bound
+            return self.serialize(s_result)
+        elif self.primitive_type == BinaryType():
+            if type(self.current_max) != bytes:
+                raise ValueError("Expected the current_max to be bytes")
+            b_result = self.current_max[: self.trunc_length]
+            if b_result != self.current_max:
+                _bytes = [*b_result]
+                for i in range(-1, -len(b_result) - 1, -1):
+                    if _bytes[i] < 255:
+                        _bytes[i] += 1
+                        return b"".join([i.to_bytes(1, byteorder="little") for i in _bytes])
+                return None
+
+            return self.serialize(b_result)
+        else:
+            return self.serialize(self.current_max)[: self.trunc_length]
+
+
+DEFAULT_TRUNCATION_LENGHT = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+
+class MetricModeTypes(Enum):
+    TRUNCATE = "truncate"
+    NONE = "none"
+    COUNTS = "counts"
+    FULL = "full"
+
+
+DEFAULT_METRICS_MODE_KEY = "write.metadata.metrics.default"
+COLUMN_METRICS_MODE_KEY_PREFIX = "write.metadata.metrics.column"
+
+
+@dataclass(frozen=True)
+class MetricsMode(Singleton):
+    type: MetricModeTypes
+    length: Optional[int] = None
+
+
+def match_metrics_mode(mode: str) -> MetricsMode:
+    m = re.match(TRUNCATION_EXPR, mode, re.IGNORECASE)
+    if m:
+        length = int(m[1])
+        if length < 1:
+            raise ValueError("Truncation length must be larger than 0")
+        return MetricsMode(MetricModeTypes.TRUNCATE, int(m[1]))
+    elif re.match("^none$", mode, re.IGNORECASE):
+        return MetricsMode(MetricModeTypes.NONE)
+    elif re.match("^counts$", mode, re.IGNORECASE):
+        return MetricsMode(MetricModeTypes.COUNTS)
+    elif re.match("^full$", mode, re.IGNORECASE):
+        return MetricsMode(MetricModeTypes.FULL)
+    else:
+        raise ValueError(f"Unsupported metrics mode {mode}")
+
+
+@dataclass(frozen=True)
+class StatisticsCollector:
+    field_id: int
+    iceberg_type: PrimitiveType
+    mode: MetricsMode
+    column_name: str
+
+
+class PyArrowStatisticsCollector(PreOrderSchemaVisitor[List[StatisticsCollector]]):
+    _field_id = 0
+    _schema: Schema
+    _properties: Dict[str, str]
+
+    def __init__(self, schema: Schema, properties: Dict[str, str]):
+        self._schema = schema
+        self._properties = properties
+
+    def schema(self, schema: Schema, struct_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        return struct_result()
+
+    def struct(
+        self, struct: StructType, field_results: List[Callable[[], List[StatisticsCollector]]]
+    ) -> List[StatisticsCollector]:
+        return list(chain(*[result() for result in field_results]))
+
+    def field(self, field: NestedField, field_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = field.field_id
+        result = field_result()
+        return result
+
+    def list(self, list_type: ListType, element_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = list_type.element_id
+        return element_result()
+
+    def map(
+        self,
+        map_type: MapType,
+        key_result: Callable[[], List[StatisticsCollector]],
+        value_result: Callable[[], List[StatisticsCollector]],
+    ) -> List[StatisticsCollector]:
+        self._field_id = map_type.key_id
+        k = key_result()
+        self._field_id = map_type.value_id
+        v = value_result()
+        return k + v
+
+    def primitive(self, primitive: PrimitiveType) -> List[StatisticsCollector]:
+        column_name = self._schema.find_column_name(self._field_id)
+        if column_name is None:
+            raise ValueError(f"Column for field {self._field_id} not found")
+
+        metrics_mode = MetricsMode(MetricModeTypes.TRUNCATE, DEFAULT_TRUNCATION_LENGHT)

Review Comment:
   ```suggestion
           metrics_mode = MetricsMode(MetricModeTypes.TRUNCATE, DEFAULT_TRUNCATION_LENGTH)
   ```



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1040,337 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = ["BOOLEAN", "INT32", "INT64", "INT96", "FLOAT", "DOUBLE", "BYTE_ARRAY", "FIXED_LEN_BYTE_ARRAY"]

Review Comment:
   ```suggestion
   _PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] maxdebayser commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "maxdebayser (via GitHub)" <gi...@apache.org>.
maxdebayser commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1285994802


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1039,335 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.values()).union({"INT96"})
+
+
+class StatsAggregator:
+    current_min: Any
+    current_max: Any
+    trunc_length: Optional[int]
+
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min = None
+        self.current_max = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        expected_physical_type = _PRIMITIVE_TO_PHYSICAL[iceberg_type]
+        if expected_physical_type != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {expected_physical_type}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        self.current_min = val if self.current_min is None else min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        self.current_max = val if self.current_max is None else max(val, self.current_max)
+
+    def get_min(self) -> bytes:
+        return self.serialize(
+            self.current_min
+            if self.trunc_length is None
+            else TruncateTransform(width=self.trunc_length).transform(self.primitive_type)(self.current_min)
+        )
+
+    def get_max(self) -> Optional[bytes]:
+        if self.current_max is None:
+            return None
+
+        if self.primitive_type == StringType():
+            if type(self.current_max) != str:
+                raise ValueError("Expected the current_max to be a string")
+
+            s_result = self.current_max[: self.trunc_length]
+            if s_result != self.current_max:
+                chars = [*s_result]
+
+                for i in range(-1, -len(s_result) - 1, -1):
+                    try:
+                        to_inc = ord(chars[i])
+                        # will raise exception if the highest unicode code is reached
+                        _next = chr(to_inc + 1)
+                        chars[i] = _next
+                        return self.serialize("".join(chars))
+                    except ValueError:
+                        pass
+                return None  # didn't find a valid upper bound
+            return self.serialize(s_result)
+        elif self.primitive_type == BinaryType():
+            if type(self.current_max) != bytes:
+                raise ValueError("Expected the current_max to be bytes")
+            b_result = self.current_max[: self.trunc_length]
+            if b_result != self.current_max:
+                _bytes = [*b_result]
+                for i in range(-1, -len(b_result) - 1, -1):
+                    if _bytes[i] < 255:
+                        _bytes[i] += 1
+                        return b"".join([i.to_bytes(1, byteorder="little") for i in _bytes])
+                return None
+
+            return self.serialize(b_result)
+        else:
+            return self.serialize(self.current_max)[: self.trunc_length]
+
+
+DEFAULT_TRUNCATION_LENGTH = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"

Review Comment:
   I'm trying to follow the behavior of the Java codebase, and I couldn't find a trim being used anywhere here  https://github.com/apache/iceberg/blob/6b1c9f0a398235b56977e817eb5d3d9457c160b5/core/src/main/java/org/apache/iceberg/MetricsModes.java#L39C21-L39C21 and here https://github.com/apache/iceberg/blob/6b1c9f0a398235b56977e817eb5d3d9457c160b5/core/src/main/java/org/apache/iceberg/MetricsConfig.java#L194
   
   But we can add it if you prefer.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] maxdebayser commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "maxdebayser (via GitHub)" <gi...@apache.org>.
maxdebayser commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1286039886


##########
python/tests/io/test_pyarrow.py:
##########
@@ -1345,3 +1374,655 @@ def test_pyarrow_wrap_fsspec(example_task: FileScanTask, table_schema_simple: Sc
 bar: [[1,2,3]]
 baz: [[true,false,null]]"""
     )
+
+
+def construct_test_table() -> Tuple[Any, Any, Union[TableMetadataV1, TableMetadataV2]]:
+    table_metadata = {
+        "format-version": 2,
+        "location": "s3://bucket/test/location",
+        "last-column-id": 7,
+        "current-schema-id": 0,
+        "schemas": [
+            {
+                "type": "struct",
+                "schema-id": 0,
+                "fields": [
+                    {"id": 1, "name": "strings", "required": False, "type": "string"},
+                    {"id": 2, "name": "floats", "required": False, "type": "float"},
+                    {
+                        "id": 3,
+                        "name": "list",
+                        "required": False,
+                        "type": {"type": "list", "element-id": 5, "element": "long", "element-required": False},
+                    },
+                    {
+                        "id": 4,
+                        "name": "maps",
+                        "required": False,
+                        "type": {
+                            "type": "map",
+                            "key-id": 6,
+                            "key": "long",
+                            "value-id": 7,
+                            "value": "long",
+                            "value-required": False,
+                        },
+                    },
+                ],
+            },
+        ],
+        "default-spec-id": 0,
+        "partition-specs": [{"spec-id": 0, "fields": []}],
+        "properties": {},
+    }
+
+    table_metadata = TableMetadataUtil.parse_obj(table_metadata)
+    arrow_schema = schema_to_pyarrow(table_metadata.schemas[0])
+
+    _strings = ["zzzzzzzzzzzzzzzzzzzz", "rrrrrrrrrrrrrrrrrrrr", None, "aaaaaaaaaaaaaaaaaaaa"]
+
+    _floats = [3.14, math.nan, 1.69, 100]
+
+    _list = [[1, 2, 3], [4, 5, 6], None, [7, 8, 9]]
+
+    _maps: List[Optional[Dict[int, int]]] = [
+        {1: 2, 3: 4},
+        None,
+        {5: 6},
+        {},
+    ]
+
+    table = pa.Table.from_pydict(
+        {
+            "strings": _strings,
+            "floats": _floats,
+            "list": _list,
+            "maps": _maps,
+        },
+        schema=arrow_schema,
+    )
+    metadata_collector: List[Any] = []
+
+    with pa.BufferOutputStream() as f:
+        with pq.ParquetWriter(f, table.schema, metadata_collector=metadata_collector) as writer:
+            writer.write_table(table)
+
+        return f.getvalue(), metadata_collector[0], table_metadata
+
+
+def test_record_count() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.record_count == 4
+
+
+def test_file_size() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.file_size_in_bytes == len(file_bytes)
+
+
+def test_value_counts() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.value_counts) == 5
+    assert datafile.value_counts[1] == 4
+    assert datafile.value_counts[2] == 4
+    assert datafile.value_counts[5] == 10  # 3 lists with 3 items and a None value
+    assert datafile.value_counts[6] == 5
+    assert datafile.value_counts[7] == 5
+
+
+def test_column_sizes() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.column_sizes) == 5
+    # these values are an artifact of how the write_table encodes the columns
+    assert datafile.column_sizes[1] == 116
+    assert datafile.column_sizes[2] == 89
+    assert datafile.column_sizes[5] == 151
+    assert datafile.column_sizes[6] == 117
+    assert datafile.column_sizes[7] == 117
+
+
+def test_null_and_nan_counts() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.null_value_counts) == 5
+    assert datafile.null_value_counts[1] == 1
+    assert datafile.null_value_counts[2] == 0
+    assert datafile.null_value_counts[5] == 1
+    assert datafile.null_value_counts[6] == 2
+    assert datafile.null_value_counts[7] == 2
+
+    # #arrow does not include this in the statistics
+    # assert len(datafile.nan_value_counts)  == 3
+    # assert datafile.nan_value_counts[1]    == 0
+    # assert datafile.nan_value_counts[2]    == 1
+    # assert datafile.nan_value_counts[3]    == 0
+
+
+def test_bounds() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.lower_bounds) == 5
+    assert datafile.lower_bounds[1].decode() == "aaaaaaaaaaaaaaaa"
+    assert datafile.lower_bounds[2] == STRUCT_FLOAT.pack(1.69)
+    assert datafile.lower_bounds[5] == STRUCT_INT64.pack(1)
+    assert datafile.lower_bounds[6] == STRUCT_INT64.pack(1)
+    assert datafile.lower_bounds[7] == STRUCT_INT64.pack(2)
+
+    assert len(datafile.upper_bounds) == 5
+    assert datafile.upper_bounds[1].decode() == "zzzzzzzzzzzzzzz{"
+    assert datafile.upper_bounds[2] == STRUCT_FLOAT.pack(100)
+    assert datafile.upper_bounds[5] == STRUCT_INT64.pack(9)
+    assert datafile.upper_bounds[6] == STRUCT_INT64.pack(5)
+    assert datafile.upper_bounds[7] == STRUCT_INT64.pack(6)
+
+
+def test_metrics_mode_parsing() -> None:
+    assert match_metrics_mode("none") == MetricsMode(MetricModeTypes.NONE)
+    assert match_metrics_mode("nOnE") == MetricsMode(MetricModeTypes.NONE)
+    assert match_metrics_mode("counts") == MetricsMode(MetricModeTypes.COUNTS)
+    assert match_metrics_mode("Counts") == MetricsMode(MetricModeTypes.COUNTS)
+    assert match_metrics_mode("full") == MetricsMode(MetricModeTypes.FULL)
+    assert match_metrics_mode("FuLl") == MetricsMode(MetricModeTypes.FULL)
+
+    with pytest.raises(ValueError) as exc_info:
+        match_metrics_mode(" Full")
+    assert "Unsupported metrics mode:  Full" in str(exc_info.value)
+
+    assert match_metrics_mode("truncate(16)") == MetricsMode(MetricModeTypes.TRUNCATE, 16)
+    assert match_metrics_mode("trUncatE(16)") == MetricsMode(MetricModeTypes.TRUNCATE, 16)
+    assert match_metrics_mode("trUncatE(7)") == MetricsMode(MetricModeTypes.TRUNCATE, 7)
+    assert match_metrics_mode("trUncatE(07)") == MetricsMode(MetricModeTypes.TRUNCATE, 7)
+
+    with pytest.raises(ValueError) as exc_info:
+        match_metrics_mode("trUncatE(-7)")
+    assert "Malformed truncate: trUncatE(-7)" in str(exc_info.value)
+
+    with pytest.raises(ValueError) as exc_info:
+        match_metrics_mode("trUncatE(0)")
+    assert "Truncation length must be larger than 0" in str(exc_info.value)
+
+
+def test_metrics_mode_none() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    table_metadata.properties["write.metadata.metrics.default"] = "none"
+    fill_parquet_file_metadata(
+        datafile,
+        metadata,
+        len(file_bytes),
+        table_metadata,
+    )
+
+    assert len(datafile.value_counts) == 0
+    assert len(datafile.null_value_counts) == 0
+    assert len(datafile.nan_value_counts) == 0
+    assert len(datafile.lower_bounds) == 0
+    assert len(datafile.upper_bounds) == 0
+
+
+def test_metrics_mode_counts() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    table_metadata.properties["write.metadata.metrics.default"] = "counts"
+    fill_parquet_file_metadata(
+        datafile,
+        metadata,
+        len(file_bytes),
+        table_metadata,
+    )
+
+    assert len(datafile.value_counts) == 5
+    assert len(datafile.null_value_counts) == 5
+    assert len(datafile.nan_value_counts) == 0
+    assert len(datafile.lower_bounds) == 0
+    assert len(datafile.upper_bounds) == 0
+
+
+def test_metrics_mode_full() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    table_metadata.properties["write.metadata.metrics.default"] = "full"
+    fill_parquet_file_metadata(
+        datafile,
+        metadata,
+        len(file_bytes),
+        table_metadata,
+    )
+
+    assert len(datafile.value_counts) == 5
+    assert len(datafile.null_value_counts) == 5
+    assert len(datafile.nan_value_counts) == 0
+
+    assert len(datafile.lower_bounds) == 5
+    assert datafile.lower_bounds[1].decode() == "aaaaaaaaaaaaaaaaaaaa"
+    assert datafile.lower_bounds[2] == STRUCT_FLOAT.pack(1.69)
+    assert datafile.lower_bounds[5] == STRUCT_INT64.pack(1)
+    assert datafile.lower_bounds[6] == STRUCT_INT64.pack(1)
+    assert datafile.lower_bounds[7] == STRUCT_INT64.pack(2)
+
+    assert len(datafile.upper_bounds) == 5
+    assert datafile.upper_bounds[1].decode() == "zzzzzzzzzzzzzzzzzzzz"
+    assert datafile.upper_bounds[2] == STRUCT_FLOAT.pack(100)
+    assert datafile.upper_bounds[5] == STRUCT_INT64.pack(9)
+    assert datafile.upper_bounds[6] == STRUCT_INT64.pack(5)
+    assert datafile.upper_bounds[7] == STRUCT_INT64.pack(6)
+
+
+def test_metrics_mode_non_default_trunc() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    table_metadata.properties["write.metadata.metrics.default"] = "truncate(2)"
+    fill_parquet_file_metadata(
+        datafile,
+        metadata,
+        len(file_bytes),
+        table_metadata,
+    )
+
+    assert len(datafile.value_counts) == 5
+    assert len(datafile.null_value_counts) == 5
+    assert len(datafile.nan_value_counts) == 0
+
+    assert len(datafile.lower_bounds) == 5
+    assert datafile.lower_bounds[1].decode() == "aa"
+    assert datafile.lower_bounds[2] == STRUCT_FLOAT.pack(1.69)
+    assert datafile.lower_bounds[5] == STRUCT_INT64.pack(1)
+    assert datafile.lower_bounds[6] == STRUCT_INT64.pack(1)
+    assert datafile.lower_bounds[7] == STRUCT_INT64.pack(2)
+
+    assert len(datafile.upper_bounds) == 5
+    assert datafile.upper_bounds[1].decode() == "z{"
+    assert datafile.upper_bounds[2] == STRUCT_FLOAT.pack(100)
+    assert datafile.upper_bounds[5] == STRUCT_INT64.pack(9)
+    assert datafile.upper_bounds[6] == STRUCT_INT64.pack(5)
+    assert datafile.upper_bounds[7] == STRUCT_INT64.pack(6)
+
+
+def test_column_metrics_mode() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    table_metadata.properties["write.metadata.metrics.default"] = "truncate(2)"
+    table_metadata.properties["write.metadata.metrics.column.strings"] = "none"
+    table_metadata.properties["write.metadata.metrics.column.list.element"] = "counts"
+    fill_parquet_file_metadata(
+        datafile,
+        metadata,
+        len(file_bytes),
+        table_metadata,
+    )
+
+    assert len(datafile.value_counts) == 4
+    assert len(datafile.null_value_counts) == 4
+    assert len(datafile.nan_value_counts) == 0
+
+    assert len(datafile.lower_bounds) == 3
+    assert datafile.lower_bounds[2] == STRUCT_FLOAT.pack(1.69)
+    assert 5 not in datafile.lower_bounds

Review Comment:
   Wait, there's not integer being truncated here.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] maxdebayser commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "maxdebayser (via GitHub)" <gi...@apache.org>.
maxdebayser commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1289036564


##########
python/tests/io/test_pyarrow.py:
##########
@@ -1345,3 +1374,655 @@ def test_pyarrow_wrap_fsspec(example_task: FileScanTask, table_schema_simple: Sc
 bar: [[1,2,3]]
 baz: [[true,false,null]]"""
     )
+
+
+def construct_test_table() -> Tuple[Any, Any, Union[TableMetadataV1, TableMetadataV2]]:
+    table_metadata = {
+        "format-version": 2,
+        "location": "s3://bucket/test/location",
+        "last-column-id": 7,
+        "current-schema-id": 0,
+        "schemas": [
+            {
+                "type": "struct",
+                "schema-id": 0,
+                "fields": [
+                    {"id": 1, "name": "strings", "required": False, "type": "string"},
+                    {"id": 2, "name": "floats", "required": False, "type": "float"},
+                    {
+                        "id": 3,
+                        "name": "list",
+                        "required": False,
+                        "type": {"type": "list", "element-id": 5, "element": "long", "element-required": False},
+                    },
+                    {
+                        "id": 4,
+                        "name": "maps",
+                        "required": False,
+                        "type": {
+                            "type": "map",
+                            "key-id": 6,
+                            "key": "long",
+                            "value-id": 7,
+                            "value": "long",
+                            "value-required": False,
+                        },
+                    },
+                ],
+            },
+        ],
+        "default-spec-id": 0,
+        "partition-specs": [{"spec-id": 0, "fields": []}],
+        "properties": {},
+    }
+
+    table_metadata = TableMetadataUtil.parse_obj(table_metadata)
+    arrow_schema = schema_to_pyarrow(table_metadata.schemas[0])
+
+    _strings = ["zzzzzzzzzzzzzzzzzzzz", "rrrrrrrrrrrrrrrrrrrr", None, "aaaaaaaaaaaaaaaaaaaa"]
+
+    _floats = [3.14, math.nan, 1.69, 100]
+
+    _list = [[1, 2, 3], [4, 5, 6], None, [7, 8, 9]]
+
+    _maps: List[Optional[Dict[int, int]]] = [
+        {1: 2, 3: 4},
+        None,
+        {5: 6},
+        {},
+    ]
+
+    table = pa.Table.from_pydict(

Review Comment:
   I've added a struct now. The value_counts and counts are a little bit weird, but they make more sense than for the lists.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] maxdebayser commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "maxdebayser (via GitHub)" <gi...@apache.org>.
maxdebayser commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1260042879


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1013,3 +1027,359 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+BOUND_TRUNCATED_LENGHT = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+logger = logging.getLogger(__name__)
+
+# Serialization rules: https://iceberg.apache.org/spec/#binary-single-value-serialization
+#
+# Type      Binary serialization
+# boolean   0x00 for false, non-zero byte for true
+# int       Stored as 4-byte little-endian
+# long      Stored as 8-byte little-endian
+# float     Stored as 4-byte little-endian
+# double    Stored as 8-byte little-endian
+# date      Stores days from the 1970-01-01 in an 4-byte little-endian int
+# time      Stores microseconds from midnight in an 8-byte little-endian long
+# timestamp without zone	Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long
+# timestamp with zone	Stores microseconds from 1970-01-01 00:00:00.000000 UTC in an 8-byte little-endian long
+# string    UTF-8 bytes (without length)
+# uuid      16-byte big-endian value, see example in Appendix B
+# fixed(L)  Binary value
+# binary    Binary value (without length)
+#
+
+
+def bool_to_avro(value: bool) -> bytes:
+    return STRUCT_BOOL.pack(value)
+
+
+def int32_to_avro(value: int) -> bytes:
+    return STRUCT_INT32.pack(value)
+
+
+def int64_to_avro(value: int) -> bytes:
+    return STRUCT_INT64.pack(value)
+
+
+def float_to_avro(value: float) -> bytes:
+    return STRUCT_FLOAT.pack(value)
+
+
+def double_to_avro(value: float) -> bytes:
+    return STRUCT_DOUBLE.pack(value)
+
+
+def bytes_to_avro(value: Union[bytes, str]) -> bytes:
+    if type(value) == str:
+        return value.encode()
+    else:
+        assert isinstance(value, bytes)  # appeases mypy
+        return value
+
+
+class StatsAggregator:
+    def __init__(self, type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.serialize: Any = None
+        self.trunc_lenght = trunc_length
+
+        if type_string == "BOOLEAN":
+            self.serialize = bool_to_avro
+        elif type_string == "INT32":
+            self.serialize = int32_to_avro
+        elif type_string == "INT64":
+            self.serialize = int64_to_avro
+        elif type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+        elif type_string == "FLOAT":
+            self.serialize = float_to_avro
+        elif type_string == "DOUBLE":
+            self.serialize = double_to_avro
+        elif type_string == "BYTE_ARRAY":
+            self.serialize = bytes_to_avro
+        elif type_string == "FIXED_LEN_BYTE_ARRAY":
+            self.serialize = bytes_to_avro
+        else:
+            raise AssertionError(f"Unknown physical type {type_string}")
+
+    def add_min(self, val: bytes) -> None:
+        if self.current_min is None:
+            self.current_min = val
+        else:
+            self.current_min = min(val, self.current_min)
+
+    def add_max(self, val: bytes) -> None:
+        if self.current_max is None:
+            self.current_max = val
+        else:
+            self.current_max = max(self.current_max, val)
+
+    def get_min(self) -> bytes:
+        return self.serialize(self.current_min)[: self.trunc_lenght]
+
+    def get_max(self) -> bytes:
+        return self.serialize(self.current_max)[: self.trunc_lenght]
+
+
+class MetricsMode(Enum):
+    NONE = 0
+    COUNTS = 1
+    TRUNC = 2

Review Comment:
   Actually now I'm seeing how it would fit into the StatisticsCollector visitor. It makes sense, let me implement your suggestion.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1263156291


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1013,3 +1025,271 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+class StatsAggregator:
+    def __init__(self, type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.trunc_length = trunc_length
+        self.primitive_type: Optional[PrimitiveType] = None
+
+        if type_string == "BOOLEAN":
+            self.primitive_type = BooleanType()
+        elif type_string == "INT32":
+            self.primitive_type = IntegerType()
+        elif type_string == "INT64":
+            self.primitive_type = LongType()
+        elif type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+        elif type_string == "FLOAT":
+            self.primitive_type = FloatType()
+        elif type_string == "DOUBLE":
+            self.primitive_type = DoubleType()
+        elif type_string == "BYTE_ARRAY":
+            self.primitive_type = BinaryType()
+        elif type_string == "FIXED_LEN_BYTE_ARRAY":
+            self.primitive_type = BinaryType()
+        else:
+            raise AssertionError(f"Unknown physical type {type_string}")
+
+    def serialize(self, value: Any) -> bytes:
+        if type(value) == str:
+            value = value.encode()
+        assert self.primitive_type is not None  # appease mypy
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        if self.current_min is None:
+            self.current_min = val
+        else:
+            self.current_min = min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        if self.current_max is None:
+            self.current_max = val
+        else:
+            self.current_max = max(self.current_max, val)
+
+    def get_min(self) -> bytes:
+        return self.serialize(self.current_min)[: self.trunc_length]
+
+    def get_max(self) -> bytes:
+        return self.serialize(self.current_max)[: self.trunc_length]

Review Comment:
   Truncation is more complicated because you can corrupt the max by truncating. We'll need to implement the equivalent of `UnicodeUtil` for max.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] maxdebayser commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "maxdebayser (via GitHub)" <gi...@apache.org>.
maxdebayser commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1261671941


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1013,3 +1027,359 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+BOUND_TRUNCATED_LENGHT = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+logger = logging.getLogger(__name__)
+
+# Serialization rules: https://iceberg.apache.org/spec/#binary-single-value-serialization
+#
+# Type      Binary serialization
+# boolean   0x00 for false, non-zero byte for true
+# int       Stored as 4-byte little-endian
+# long      Stored as 8-byte little-endian
+# float     Stored as 4-byte little-endian
+# double    Stored as 8-byte little-endian
+# date      Stores days from the 1970-01-01 in an 4-byte little-endian int
+# time      Stores microseconds from midnight in an 8-byte little-endian long
+# timestamp without zone	Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long
+# timestamp with zone	Stores microseconds from 1970-01-01 00:00:00.000000 UTC in an 8-byte little-endian long
+# string    UTF-8 bytes (without length)
+# uuid      16-byte big-endian value, see example in Appendix B
+# fixed(L)  Binary value
+# binary    Binary value (without length)
+#
+
+
+def bool_to_avro(value: bool) -> bytes:
+    return STRUCT_BOOL.pack(value)
+
+
+def int32_to_avro(value: int) -> bytes:
+    return STRUCT_INT32.pack(value)
+
+
+def int64_to_avro(value: int) -> bytes:
+    return STRUCT_INT64.pack(value)
+
+
+def float_to_avro(value: float) -> bytes:
+    return STRUCT_FLOAT.pack(value)
+
+
+def double_to_avro(value: float) -> bytes:
+    return STRUCT_DOUBLE.pack(value)
+
+
+def bytes_to_avro(value: Union[bytes, str]) -> bytes:
+    if type(value) == str:
+        return value.encode()
+    else:
+        assert isinstance(value, bytes)  # appeases mypy
+        return value
+
+
+class StatsAggregator:
+    def __init__(self, type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.serialize: Any = None
+        self.trunc_lenght = trunc_length
+
+        if type_string == "BOOLEAN":
+            self.serialize = bool_to_avro
+        elif type_string == "INT32":
+            self.serialize = int32_to_avro
+        elif type_string == "INT64":
+            self.serialize = int64_to_avro
+        elif type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+        elif type_string == "FLOAT":
+            self.serialize = float_to_avro
+        elif type_string == "DOUBLE":
+            self.serialize = double_to_avro
+        elif type_string == "BYTE_ARRAY":
+            self.serialize = bytes_to_avro
+        elif type_string == "FIXED_LEN_BYTE_ARRAY":
+            self.serialize = bytes_to_avro
+        else:
+            raise AssertionError(f"Unknown physical type {type_string}")
+
+    def add_min(self, val: bytes) -> None:
+        if self.current_min is None:
+            self.current_min = val
+        else:
+            self.current_min = min(val, self.current_min)
+
+    def add_max(self, val: bytes) -> None:
+        if self.current_max is None:
+            self.current_max = val
+        else:
+            self.current_max = max(self.current_max, val)
+
+    def get_min(self) -> bytes:
+        return self.serialize(self.current_min)[: self.trunc_lenght]
+
+    def get_max(self) -> bytes:
+        return self.serialize(self.current_max)[: self.trunc_lenght]
+
+
+class MetricsMode(Enum):
+    NONE = 0
+    COUNTS = 1
+    TRUNC = 2
+    FULL = 3
+
+
+def match_metrics_mode(mode: str) -> Tuple[MetricsMode, Optional[int]]:
+    m = re.match(TRUNCATION_EXPR, mode)
+
+    if m:
+        return MetricsMode.TRUNC, int(m[1])
+    elif mode == "none":
+        return MetricsMode.NONE, None
+    elif mode == "counts":
+        return MetricsMode.COUNTS, None
+    elif mode == "full":
+        return MetricsMode.FULL, None
+    else:
+        raise AssertionError(f"Unsupported metrics mode {mode}")
+
+
+def fill_parquet_file_metadata(
+    df: DataFile,
+    metadata: pq.FileMetaData,
+    col_path_2_iceberg_id: Dict[str, int],
+    file_size: int,
+    table_metadata: Optional[TableMetadata] = None,
+) -> None:
+    """
+    Computes and fills the following fields of the DataFile object.
+
+    - file_format
+    - record_count
+    - file_size_in_bytes
+    - column_sizes
+    - value_counts
+    - null_value_counts
+    - nan_value_counts
+    - lower_bounds
+    - upper_bounds
+    - split_offsets
+
+    Args:
+        df (DataFile): A DataFile object representing the Parquet file for which metadata is to be filled.
+        metadata (pyarrow.parquet.FileMetaData): A pyarrow metadata object.
+        col_path_2_iceberg_id: A mapping of column paths as in the `path_in_schema` attribute of the colum
+            metadata to iceberg schema IDs. For scalar columns this will be the column name. For complex types
+            it could be something like `my_map.key_value.value`
+        file_size (int): The total compressed file size cannot be retrieved from the metadata and hence has to
+            be passed here. Depending on the kind of file system and pyarrow library call used, different
+            ways to obtain this value might be appropriate.
+    """
+    col_index_2_id = {}
+
+    col_names = {p.split(".")[0] for p in col_path_2_iceberg_id.keys()}
+
+    metrics_modes = {n: MetricsMode.TRUNC for n in col_names}
+    trunc_lengths: Dict[str, Optional[int]] = {n: BOUND_TRUNCATED_LENGHT for n in col_names}
+
+    if table_metadata:
+        default_mode = table_metadata.properties.get("write.metadata.metrics.default")
+
+        if default_mode:
+            m, t = match_metrics_mode(default_mode)
+
+            metrics_modes = {n: m for n in col_names}
+            trunc_lengths = {n: t for n in col_names}
+
+        for col_name in col_names:
+            col_mode = table_metadata.properties.get(f"write.metadata.metrics.column.{col_name}")

Review Comment:
   I couldn't find a lot of good examples of the Java API, so I'm assuming that the `Schema.find_column_name()` returns a fully qualified column name that is consistent with the Java API.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1267878112


##########
python/tests/io/test_pyarrow.py:
##########
@@ -1330,3 +1342,411 @@ def test_pyarrow_wrap_fsspec(example_task: FileScanTask, table_schema_simple: Sc
 bar: [[1,2,3]]
 baz: [[true,false,null]]"""
     )
+
+
+def construct_test_table() -> pa.Buffer:
+    table_metadata = {
+        "format-version": 2,
+        "location": "s3://bucket/test/location",
+        "last-column-id": 7,
+        "current-schema-id": 0,
+        "schemas": [
+            {
+                "type": "struct",
+                "schema-id": 0,
+                "fields": [
+                    {"id": 1, "name": "strings", "required": False, "type": "string"},
+                    {"id": 2, "name": "floats", "required": False, "type": "float"},
+                    {
+                        "id": 3,
+                        "name": "list",
+                        "required": False,
+                        "type": {"type": "list", "element-id": 5, "element": "long", "element-required": False},
+                    },
+                    {
+                        "id": 4,
+                        "name": "maps",
+                        "required": False,
+                        "type": {
+                            "type": "map",
+                            "key-id": 6,
+                            "key": "long",
+                            "value-id": 7,
+                            "value": "long",
+                            "value-required": False,
+                        },
+                    },
+                ],
+            },
+        ],
+        "default-spec-id": 0,
+        "partition-specs": [{"spec-id": 0, "fields": []}],
+        "properties": {},
+    }
+
+    table_metadata = TableMetadataUtil.parse_obj(table_metadata)
+    arrow_schema = schema_to_pyarrow(table_metadata.schemas[0])
+
+    _strings = ["zzzzzzzzzzzzzzzzzzzz", "rrrrrrrrrrrrrrrrrrrr", None, "aaaaaaaaaaaaaaaaaaaa"]
+
+    _floats = [3.14, math.nan, 1.69, 100]
+
+    _list = [[1, 2, 3], [4, 5, 6], None, [7, 8, 9]]
+
+    _maps: List[Optional[Dict[int, int]]] = [
+        {1: 2, 3: 4},
+        None,
+        {5: 6},
+        {},
+    ]
+
+    table = pa.Table.from_pydict(
+        {
+            "strings": _strings,
+            "floats": _floats,
+            "list": _list,
+            "maps": _maps,
+        },
+        schema=arrow_schema,
+    )
+    f = pa.BufferOutputStream()
+
+    metadata_collector: List[Any] = []
+    writer = pq.ParquetWriter(f, table.schema, metadata_collector=metadata_collector)
+
+    writer.write_table(table)
+    writer.close()
+
+    return f.getvalue(), metadata_collector[0], table_metadata
+
+
+def test_record_count() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.record_count == 4
+
+
+def test_file_size() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.file_size_in_bytes == len(file_bytes)
+
+
+def test_value_counts() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.value_counts) == 5
+    assert datafile.value_counts[1] == 4
+    assert datafile.value_counts[2] == 4
+    assert datafile.value_counts[5] == 10  # 3 lists with 3 items and a None value
+    assert datafile.value_counts[6] == 5
+    assert datafile.value_counts[7] == 5
+
+
+def test_column_sizes() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.column_sizes) == 5
+    # these values are an artifact of how the write_table encodes the columns
+    assert datafile.column_sizes[1] == 116
+    assert datafile.column_sizes[2] == 89
+    assert datafile.column_sizes[5] == 151
+    assert datafile.column_sizes[6] == 117
+    assert datafile.column_sizes[7] == 117
+
+
+def test_null_and_nan_counts() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.null_value_counts) == 5
+    assert datafile.null_value_counts[1] == 1
+    assert datafile.null_value_counts[2] == 0
+    assert datafile.null_value_counts[5] == 1
+    assert datafile.null_value_counts[6] == 2
+    assert datafile.null_value_counts[7] == 2
+
+    # #arrow does not include this in the statistics
+    # assert len(datafile.nan_value_counts)  == 3
+    # assert datafile.nan_value_counts[1]    == 0
+    # assert datafile.nan_value_counts[2]    == 1
+    # assert datafile.nan_value_counts[3]    == 0
+
+
+def test_bounds() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.lower_bounds) == 5
+    assert datafile.lower_bounds[1].decode() == "aaaaaaaaaaaaaaaaaaaa"[:DEFAULT_TRUNCATION_LENGHT]
+    assert datafile.lower_bounds[2] == STRUCT_FLOAT.pack(1.69)
+    assert datafile.lower_bounds[5] == STRUCT_INT64.pack(1)
+    assert datafile.lower_bounds[6] == STRUCT_INT64.pack(1)
+    assert datafile.lower_bounds[7] == STRUCT_INT64.pack(2)
+
+    assert len(datafile.upper_bounds) == 5
+    assert datafile.upper_bounds[1].decode() == "zzzzzzzzzzzzzzzzzzzz"[:DEFAULT_TRUNCATION_LENGHT]
+    assert datafile.upper_bounds[2] == STRUCT_FLOAT.pack(100)
+    assert datafile.upper_bounds[5] == STRUCT_INT64.pack(9)
+    assert datafile.upper_bounds[6] == STRUCT_INT64.pack(5)
+    assert datafile.upper_bounds[7] == STRUCT_INT64.pack(6)
+
+
+def test_metrics_mode_parsing() -> None:
+    assert match_metrics_mode("none") == MetricsMode(MetricModeTypes.NONE)
+    assert match_metrics_mode("nOnE") == MetricsMode(MetricModeTypes.NONE)
+    assert match_metrics_mode("counts") == MetricsMode(MetricModeTypes.COUNTS)
+    assert match_metrics_mode("Counts") == MetricsMode(MetricModeTypes.COUNTS)
+    assert match_metrics_mode("full") == MetricsMode(MetricModeTypes.FULL)
+    assert match_metrics_mode("FuLl") == MetricsMode(MetricModeTypes.FULL)
+
+    with pytest.raises(AssertionError) as exc_info:
+        match_metrics_mode(" Full")
+    assert "Unsupported metrics mode  Full" in str(exc_info.value)
+
+    assert match_metrics_mode("truncate(16)") == MetricsMode(MetricModeTypes.TRUNCATE, 16)
+    assert match_metrics_mode("trUncatE(16)") == MetricsMode(MetricModeTypes.TRUNCATE, 16)
+    assert match_metrics_mode("trUncatE(7)") == MetricsMode(MetricModeTypes.TRUNCATE, 7)
+    assert match_metrics_mode("trUncatE(07)") == MetricsMode(MetricModeTypes.TRUNCATE, 7)
+
+    with pytest.raises(AssertionError) as exc_info:
+        match_metrics_mode("trUncatE(-7)")
+    assert "Unsupported metrics mode trUncatE(-7)" in str(exc_info.value)
+
+    with pytest.raises(AssertionError) as exc_info:
+        match_metrics_mode("trUncatE(0)")
+    assert "Truncation length must be larger than 0" in str(exc_info.value)
+
+
+def test_metrics_mode_none() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    table_metadata.properties["write.metadata.metrics.default"] = "none"
+    fill_parquet_file_metadata(
+        datafile,
+        metadata,
+        len(file_bytes),
+        table_metadata,
+    )
+
+    assert len(datafile.value_counts) == 0
+    assert len(datafile.null_value_counts) == 0
+    assert len(datafile.nan_value_counts) == 0
+    assert len(datafile.lower_bounds) == 0
+    assert len(datafile.upper_bounds) == 0
+
+
+def test_metrics_mode_counts() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    table_metadata.properties["write.metadata.metrics.default"] = "counts"
+    fill_parquet_file_metadata(
+        datafile,
+        metadata,
+        len(file_bytes),
+        table_metadata,
+    )
+
+    assert len(datafile.value_counts) == 5
+    assert len(datafile.null_value_counts) == 5
+    assert len(datafile.nan_value_counts) == 0
+    assert len(datafile.lower_bounds) == 0
+    assert len(datafile.upper_bounds) == 0
+
+
+def test_metrics_mode_full() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    table_metadata.properties["write.metadata.metrics.default"] = "full"
+    fill_parquet_file_metadata(
+        datafile,
+        metadata,
+        len(file_bytes),
+        table_metadata,
+    )
+
+    assert len(datafile.value_counts) == 5
+    assert len(datafile.null_value_counts) == 5
+    assert len(datafile.nan_value_counts) == 0
+
+    assert len(datafile.lower_bounds) == 5
+    assert datafile.lower_bounds[1].decode() == "aaaaaaaaaaaaaaaaaaaa"
+    assert datafile.lower_bounds[2] == STRUCT_FLOAT.pack(1.69)
+    assert datafile.lower_bounds[5] == STRUCT_INT64.pack(1)
+    assert datafile.lower_bounds[6] == STRUCT_INT64.pack(1)
+    assert datafile.lower_bounds[7] == STRUCT_INT64.pack(2)
+
+    assert len(datafile.upper_bounds) == 5
+    assert datafile.upper_bounds[1].decode() == "zzzzzzzzzzzzzzzzzzzz"
+    assert datafile.upper_bounds[2] == STRUCT_FLOAT.pack(100)
+    assert datafile.upper_bounds[5] == STRUCT_INT64.pack(9)
+    assert datafile.upper_bounds[6] == STRUCT_INT64.pack(5)
+    assert datafile.upper_bounds[7] == STRUCT_INT64.pack(6)
+
+
+def test_metrics_mode_non_default_trunc() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    table_metadata.properties["write.metadata.metrics.default"] = "truncate(2)"
+    fill_parquet_file_metadata(
+        datafile,
+        metadata,
+        len(file_bytes),
+        table_metadata,
+    )
+
+    assert len(datafile.value_counts) == 5
+    assert len(datafile.null_value_counts) == 5
+    assert len(datafile.nan_value_counts) == 0
+
+    assert len(datafile.lower_bounds) == 5
+    assert datafile.lower_bounds[1].decode() == "aa"
+    assert datafile.lower_bounds[2] == STRUCT_FLOAT.pack(1.69)[:2]
+    assert datafile.lower_bounds[5] == STRUCT_INT64.pack(1)[:2]
+    assert datafile.lower_bounds[6] == STRUCT_INT64.pack(1)[:2]
+    assert datafile.lower_bounds[7] == STRUCT_INT64.pack(2)[:2]
+
+    assert len(datafile.upper_bounds) == 5
+    assert datafile.upper_bounds[1].decode() == "zz"
+    assert datafile.upper_bounds[2] == STRUCT_FLOAT.pack(100)[:2]
+    assert datafile.upper_bounds[5] == STRUCT_INT64.pack(9)[:2]
+    assert datafile.upper_bounds[6] == STRUCT_INT64.pack(5)[:2]
+    assert datafile.upper_bounds[7] == STRUCT_INT64.pack(6)[:2]
+
+
+def test_column_metrics_mode() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    table_metadata.properties["write.metadata.metrics.default"] = "truncate(2)"
+    table_metadata.properties["write.metadata.metrics.column.strings"] = "none"
+    table_metadata.properties["write.metadata.metrics.column.list.element"] = "counts"
+    fill_parquet_file_metadata(
+        datafile,
+        metadata,
+        len(file_bytes),
+        table_metadata,
+    )
+
+    assert len(datafile.value_counts) == 4
+    assert len(datafile.null_value_counts) == 4
+    assert len(datafile.nan_value_counts) == 0
+
+    assert len(datafile.lower_bounds) == 3
+    assert datafile.lower_bounds[2] == STRUCT_FLOAT.pack(1.69)[:2]
+    assert 5 not in datafile.lower_bounds
+    assert datafile.lower_bounds[6] == STRUCT_INT64.pack(1)[:2]
+    assert datafile.lower_bounds[7] == STRUCT_INT64.pack(2)[:2]
+
+    assert len(datafile.upper_bounds) == 3
+    assert datafile.upper_bounds[2] == STRUCT_FLOAT.pack(100)[:2]
+    assert 5 not in datafile.upper_bounds
+    assert datafile.upper_bounds[6] == STRUCT_INT64.pack(5)[:2]
+    assert datafile.upper_bounds[7] == STRUCT_INT64.pack(6)[:2]
+
+
+def test_offsets() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.split_offsets is not None
+    assert len(datafile.split_offsets) == 1
+    assert datafile.split_offsets[0] == 4
+

Review Comment:
   Thanks for all the tests, this is great @maxdebayser!
   
   Could you add these two for my sanity:
   
   ```python
   def test_write_and_read_stats_schema(table_schema_nested: Schema):
       tbl = pa.Table.from_pydict({
           "foo": ["a", "b"],
           "bar": [1, 2],
           "baz": [False, True],
           "qux": [["a", "b"], ["c", "d"]],
           "quux": [[("a", (("aa", 1), ("ab", 2)))], [("b", (("ba", 3), ("bb", 4)))]],
           "location": [[(52.377956, 4.897070), (4.897070, -122.431297)],
                        [(43.618881, -116.215019), (41.881832, -87.623177)]],
           "person": [("Fokko", 33), ("Max", 42)]  # Possible data quality issue
       },
           schema=schema_to_pyarrow(table_schema_nested)
       )
       stats_columns = pre_order_visit(table_schema_nested, PyArrowStatisticsCollector(table_schema_nested, {}))
   
       visited_paths = []
   
       def file_visitor(written_file: Any) -> None:
           visited_paths.append(written_file)
   
       with tempfile.TemporaryDirectory() as tmpdir:
           pq.write_to_dataset(tbl, tmpdir, file_visitor=file_visitor)
   
       assert visited_paths[0].metadata.num_columns == len(stats_columns)
   
   def test_stats_types(table_schema_nested: Schema):
       stats_columns = pre_order_visit(table_schema_nested, PyArrowStatisticsCollector(table_schema_nested, {}))
   
       # the field-ids should be sorted
       assert all(stats_columns[i].field_id <= stats_columns[i + 1].field_id for i in range(len(stats_columns)-1))
       assert [col.iceberg_type for col in stats_columns] == [
           StringType(),
           IntegerType(),
           BooleanType(),
           StringType(),
           StringType(),
           StringType(),
           IntegerType(),
           FloatType(),
           FloatType(),
           StringType(),
           IntegerType(),
       ]
   
   ```



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1285261199


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1039,335 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.values()).union({"INT96"})
+
+
+class StatsAggregator:
+    current_min: Any
+    current_max: Any
+    trunc_length: Optional[int]
+
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min = None
+        self.current_max = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        expected_physical_type = _PRIMITIVE_TO_PHYSICAL[iceberg_type]
+        if expected_physical_type != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {expected_physical_type}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        self.current_min = val if self.current_min is None else min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        self.current_max = val if self.current_max is None else max(val, self.current_max)
+
+    def get_min(self) -> bytes:
+        return self.serialize(
+            self.current_min
+            if self.trunc_length is None
+            else TruncateTransform(width=self.trunc_length).transform(self.primitive_type)(self.current_min)
+        )
+
+    def get_max(self) -> Optional[bytes]:
+        if self.current_max is None:
+            return None
+
+        if self.primitive_type == StringType():
+            if type(self.current_max) != str:
+                raise ValueError("Expected the current_max to be a string")
+
+            s_result = self.current_max[: self.trunc_length]
+            if s_result != self.current_max:
+                chars = [*s_result]
+
+                for i in range(-1, -len(s_result) - 1, -1):
+                    try:
+                        to_inc = ord(chars[i])
+                        # will raise exception if the highest unicode code is reached
+                        _next = chr(to_inc + 1)
+                        chars[i] = _next
+                        return self.serialize("".join(chars))
+                    except ValueError:
+                        pass
+                return None  # didn't find a valid upper bound
+            return self.serialize(s_result)
+        elif self.primitive_type == BinaryType():
+            if type(self.current_max) != bytes:
+                raise ValueError("Expected the current_max to be bytes")
+            b_result = self.current_max[: self.trunc_length]
+            if b_result != self.current_max:
+                _bytes = [*b_result]
+                for i in range(-1, -len(b_result) - 1, -1):
+                    if _bytes[i] < 255:
+                        _bytes[i] += 1
+                        return b"".join([i.to_bytes(1, byteorder="little") for i in _bytes])
+                return None
+
+            return self.serialize(b_result)
+        else:
+            return self.serialize(self.current_max)[: self.trunc_length]
+
+
+DEFAULT_TRUNCATION_LENGTH = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+
+class MetricModeTypes(Enum):
+    TRUNCATE = "truncate"
+    NONE = "none"
+    COUNTS = "counts"
+    FULL = "full"
+
+
+DEFAULT_METRICS_MODE_KEY = "write.metadata.metrics.default"
+COLUMN_METRICS_MODE_KEY_PREFIX = "write.metadata.metrics.column"
+
+
+@dataclass(frozen=True)
+class MetricsMode(Singleton):
+    type: MetricModeTypes
+    length: Optional[int] = None
+
+
+def match_metrics_mode(mode: str) -> MetricsMode:
+    sanitized_mode = mode.lower()
+    if sanitized_mode.startswith("truncate"):
+        m = re.match(TRUNCATION_EXPR, mode, re.IGNORECASE)
+        if m:
+            length = int(m[1])
+            if length < 1:
+                raise ValueError("Truncation length must be larger than 0")
+            return MetricsMode(MetricModeTypes.TRUNCATE, int(m[1]))
+        else:
+            raise ValueError(f"Malformed truncate: {mode}")
+    elif sanitized_mode.startswith("none"):
+        return MetricsMode(MetricModeTypes.NONE)
+    elif sanitized_mode.startswith("counts"):
+        return MetricsMode(MetricModeTypes.COUNTS)
+    elif sanitized_mode.startswith("full"):
+        return MetricsMode(MetricModeTypes.FULL)
+    else:
+        raise ValueError(f"Unsupported metrics mode: {mode}")
+
+
+@dataclass(frozen=True)
+class StatisticsCollector:
+    field_id: int
+    iceberg_type: PrimitiveType
+    mode: MetricsMode
+    column_name: str
+
+
+class PyArrowStatisticsCollector(PreOrderSchemaVisitor[List[StatisticsCollector]]):
+    _field_id: int = 0
+    _schema: Schema
+    _properties: Dict[str, str]
+
+    def __init__(self, schema: Schema, properties: Dict[str, str]):
+        self._schema = schema
+        self._properties = properties
+
+    def schema(self, schema: Schema, struct_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        return struct_result()
+
+    def struct(
+        self, struct: StructType, field_results: List[Callable[[], List[StatisticsCollector]]]
+    ) -> List[StatisticsCollector]:
+        return list(chain(*[result() for result in field_results]))
+
+    def field(self, field: NestedField, field_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = field.field_id
+        result = field_result()
+        return result
+
+    def list(self, list_type: ListType, element_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = list_type.element_id
+        return element_result()
+
+    def map(
+        self,
+        map_type: MapType,
+        key_result: Callable[[], List[StatisticsCollector]],
+        value_result: Callable[[], List[StatisticsCollector]],
+    ) -> List[StatisticsCollector]:
+        self._field_id = map_type.key_id
+        k = key_result()
+        self._field_id = map_type.value_id
+        v = value_result()
+        return k + v
+
+    def primitive(self, primitive: PrimitiveType) -> List[StatisticsCollector]:
+        column_name = self._schema.find_column_name(self._field_id)
+        if column_name is None:
+            raise ValueError(f"Column for field {self._field_id} not found")
+
+        metrics_mode = MetricsMode(MetricModeTypes.TRUNCATE, DEFAULT_TRUNCATION_LENGTH)
+
+        default_mode = self._properties.get(DEFAULT_METRICS_MODE_KEY)
+        if default_mode:
+            metrics_mode = match_metrics_mode(default_mode)
+
+        col_mode = self._properties.get(f"{COLUMN_METRICS_MODE_KEY_PREFIX}.{column_name}")
+        if col_mode:
+            metrics_mode = match_metrics_mode(col_mode)
+
+        if (
+            not (isinstance(primitive, StringType) or isinstance(primitive, BinaryType))
+            and metrics_mode.type == MetricModeTypes.TRUNCATE
+        ):
+            metrics_mode = MetricsMode(MetricModeTypes.FULL)
+
+        return [StatisticsCollector(field_id=self._field_id, iceberg_type=primitive, mode=metrics_mode, column_name=column_name)]
+
+
+def fill_parquet_file_metadata(
+    df: DataFile,
+    parquet_metadata: pq.FileMetaData,
+    file_size: int,
+    table_metadata: TableMetadata,
+) -> None:
+    """
+    Computes and fills the following fields of the DataFile object.
+
+    - file_format
+    - record_count
+    - file_size_in_bytes
+    - column_sizes
+    - value_counts
+    - null_value_counts
+    - nan_value_counts
+    - lower_bounds
+    - upper_bounds
+    - split_offsets
+
+    Args:
+        df (DataFile): A DataFile object representing the Parquet file for which metadata is to be filled.
+        parquet_metadata (pyarrow.parquet.FileMetaData): A pyarrow metadata object.
+        file_size (int): The total compressed file size cannot be retrieved from the metadata and hence has to
+            be passed here. Depending on the kind of file system and pyarrow library call used, different
+            ways to obtain this value might be appropriate.
+        table_metadata (pyiceberg.table.metadata.TableMetadata): The Iceberg table metadata. It is required to
+            compute the mapping if column position to iceberg schema type id. It's also used to set the mode
+            for column metrics collection
+    """
+    schema = next(filter(lambda s: s.schema_id == table_metadata.current_schema_id, table_metadata.schemas))
+
+    stats_columns = pre_order_visit(schema, PyArrowStatisticsCollector(schema, table_metadata.properties))
+
+    if parquet_metadata.num_columns != len(stats_columns):
+        raise ValueError(
+            f"Number of columns in metadata ({len(stats_columns)}) is different from the number of columns in pyarrow table ({parquet_metadata.num_columns})"
+        )
+
+    column_sizes: Dict[int, int] = {}
+    value_counts: Dict[int, int] = {}
+    split_offsets: List[int] = []
+
+    null_value_counts: Dict[int, int] = {}
+    nan_value_counts: Dict[int, int] = {}
+
+    col_aggs = {}
+
+    for r in range(parquet_metadata.num_row_groups):
+        # References:
+        # https://github.com/apache/iceberg/blob/fc381a81a1fdb8f51a0637ca27cd30673bd7aad3/parquet/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java#L232
+        # https://github.com/apache/parquet-mr/blob/ac29db4611f86a07cc6877b416aa4b183e09b353/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/ColumnChunkMetaData.java#L184
+
+        row_group = parquet_metadata.row_group(r)
+
+        data_offset = row_group.column(0).data_page_offset
+        dictionary_offset = row_group.column(0).dictionary_page_offset
+
+        if row_group.column(0).has_dictionary_page and dictionary_offset < data_offset:
+            split_offsets.append(dictionary_offset)
+        else:
+            split_offsets.append(data_offset)
+
+        for pos, stats_col in enumerate(stats_columns):
+            field_id = stats_col.field_id
+
+            column = row_group.column(pos)
+
+            column_sizes[field_id] = column_sizes.get(field_id, 0) + column.total_compressed_size
+
+            if stats_col.mode == MetricsMode(MetricModeTypes.NONE):
+                continue
+
+            value_counts[field_id] = value_counts.get(field_id, 0) + column.num_values
+
+            if column.is_stats_set:
+                try:
+                    statistics = column.statistics
+
+                    null_value_counts[field_id] = null_value_counts.get(field_id, 0) + statistics.null_count
+
+                    if stats_col.mode == MetricsMode(MetricModeTypes.COUNTS):
+                        continue
+
+                    if field_id not in col_aggs:
+                        col_aggs[field_id] = StatsAggregator(
+                            stats_col.iceberg_type, statistics.physical_type, stats_col.mode.length
+                        )
+
+                    col_aggs[field_id].add_min(statistics.min)
+                    col_aggs[field_id].add_max(statistics.max)
+
+                except pyarrow.lib.ArrowNotImplementedError as e:
+                    logger.warning(e)
+            else:
+                logger.warning("PyArrow statistics missing for column %d when writing file", pos)
+
+    split_offsets.sort()
+
+    lower_bounds = {}
+    upper_bounds = {}
+
+    for k, agg in col_aggs.items():
+        lower_bounds[k] = agg.get_min()

Review Comment:
   Instead of `get_min`, it would be better to signal that this is serializing the value. What about `min_as_bytes`?



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1285262852


##########
python/tests/io/test_pyarrow.py:
##########
@@ -1345,3 +1374,655 @@ def test_pyarrow_wrap_fsspec(example_task: FileScanTask, table_schema_simple: Sc
 bar: [[1,2,3]]
 baz: [[true,false,null]]"""
     )
+
+
+def construct_test_table() -> Tuple[Any, Any, Union[TableMetadataV1, TableMetadataV2]]:
+    table_metadata = {
+        "format-version": 2,
+        "location": "s3://bucket/test/location",
+        "last-column-id": 7,
+        "current-schema-id": 0,
+        "schemas": [
+            {
+                "type": "struct",
+                "schema-id": 0,
+                "fields": [
+                    {"id": 1, "name": "strings", "required": False, "type": "string"},
+                    {"id": 2, "name": "floats", "required": False, "type": "float"},
+                    {
+                        "id": 3,
+                        "name": "list",
+                        "required": False,
+                        "type": {"type": "list", "element-id": 5, "element": "long", "element-required": False},
+                    },
+                    {
+                        "id": 4,
+                        "name": "maps",
+                        "required": False,
+                        "type": {
+                            "type": "map",
+                            "key-id": 6,
+                            "key": "long",
+                            "value-id": 7,
+                            "value": "long",
+                            "value-required": False,
+                        },
+                    },
+                ],
+            },
+        ],
+        "default-spec-id": 0,
+        "partition-specs": [{"spec-id": 0, "fields": []}],
+        "properties": {},
+    }
+
+    table_metadata = TableMetadataUtil.parse_obj(table_metadata)
+    arrow_schema = schema_to_pyarrow(table_metadata.schemas[0])
+
+    _strings = ["zzzzzzzzzzzzzzzzzzzz", "rrrrrrrrrrrrrrrrrrrr", None, "aaaaaaaaaaaaaaaaaaaa"]
+
+    _floats = [3.14, math.nan, 1.69, 100]
+
+    _list = [[1, 2, 3], [4, 5, 6], None, [7, 8, 9]]
+
+    _maps: List[Optional[Dict[int, int]]] = [
+        {1: 2, 3: 4},
+        None,
+        {5: 6},
+        {},
+    ]
+
+    table = pa.Table.from_pydict(
+        {
+            "strings": _strings,
+            "floats": _floats,
+            "list": _list,
+            "maps": _maps,
+        },
+        schema=arrow_schema,
+    )
+    metadata_collector: List[Any] = []
+
+    with pa.BufferOutputStream() as f:
+        with pq.ParquetWriter(f, table.schema, metadata_collector=metadata_collector) as writer:
+            writer.write_table(table)
+
+        return f.getvalue(), metadata_collector[0], table_metadata
+
+
+def test_record_count() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.record_count == 4
+
+
+def test_file_size() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.file_size_in_bytes == len(file_bytes)
+
+
+def test_value_counts() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.value_counts) == 5
+    assert datafile.value_counts[1] == 4
+    assert datafile.value_counts[2] == 4
+    assert datafile.value_counts[5] == 10  # 3 lists with 3 items and a None value
+    assert datafile.value_counts[6] == 5
+    assert datafile.value_counts[7] == 5
+
+
+def test_column_sizes() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.column_sizes) == 5
+    # these values are an artifact of how the write_table encodes the columns
+    assert datafile.column_sizes[1] == 116
+    assert datafile.column_sizes[2] == 89
+    assert datafile.column_sizes[5] == 151
+    assert datafile.column_sizes[6] == 117
+    assert datafile.column_sizes[7] == 117
+
+
+def test_null_and_nan_counts() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.null_value_counts) == 5
+    assert datafile.null_value_counts[1] == 1
+    assert datafile.null_value_counts[2] == 0
+    assert datafile.null_value_counts[5] == 1
+    assert datafile.null_value_counts[6] == 2
+    assert datafile.null_value_counts[7] == 2
+
+    # #arrow does not include this in the statistics
+    # assert len(datafile.nan_value_counts)  == 3
+    # assert datafile.nan_value_counts[1]    == 0
+    # assert datafile.nan_value_counts[2]    == 1
+    # assert datafile.nan_value_counts[3]    == 0
+
+
+def test_bounds() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.lower_bounds) == 5
+    assert datafile.lower_bounds[1].decode() == "aaaaaaaaaaaaaaaa"
+    assert datafile.lower_bounds[2] == STRUCT_FLOAT.pack(1.69)
+    assert datafile.lower_bounds[5] == STRUCT_INT64.pack(1)
+    assert datafile.lower_bounds[6] == STRUCT_INT64.pack(1)
+    assert datafile.lower_bounds[7] == STRUCT_INT64.pack(2)
+
+    assert len(datafile.upper_bounds) == 5
+    assert datafile.upper_bounds[1].decode() == "zzzzzzzzzzzzzzz{"
+    assert datafile.upper_bounds[2] == STRUCT_FLOAT.pack(100)
+    assert datafile.upper_bounds[5] == STRUCT_INT64.pack(9)
+    assert datafile.upper_bounds[6] == STRUCT_INT64.pack(5)
+    assert datafile.upper_bounds[7] == STRUCT_INT64.pack(6)
+
+
+def test_metrics_mode_parsing() -> None:
+    assert match_metrics_mode("none") == MetricsMode(MetricModeTypes.NONE)
+    assert match_metrics_mode("nOnE") == MetricsMode(MetricModeTypes.NONE)
+    assert match_metrics_mode("counts") == MetricsMode(MetricModeTypes.COUNTS)
+    assert match_metrics_mode("Counts") == MetricsMode(MetricModeTypes.COUNTS)
+    assert match_metrics_mode("full") == MetricsMode(MetricModeTypes.FULL)
+    assert match_metrics_mode("FuLl") == MetricsMode(MetricModeTypes.FULL)
+
+    with pytest.raises(ValueError) as exc_info:
+        match_metrics_mode(" Full")
+    assert "Unsupported metrics mode:  Full" in str(exc_info.value)
+
+    assert match_metrics_mode("truncate(16)") == MetricsMode(MetricModeTypes.TRUNCATE, 16)
+    assert match_metrics_mode("trUncatE(16)") == MetricsMode(MetricModeTypes.TRUNCATE, 16)
+    assert match_metrics_mode("trUncatE(7)") == MetricsMode(MetricModeTypes.TRUNCATE, 7)
+    assert match_metrics_mode("trUncatE(07)") == MetricsMode(MetricModeTypes.TRUNCATE, 7)
+
+    with pytest.raises(ValueError) as exc_info:
+        match_metrics_mode("trUncatE(-7)")
+    assert "Malformed truncate: trUncatE(-7)" in str(exc_info.value)
+
+    with pytest.raises(ValueError) as exc_info:
+        match_metrics_mode("trUncatE(0)")
+    assert "Truncation length must be larger than 0" in str(exc_info.value)
+
+
+def test_metrics_mode_none() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    table_metadata.properties["write.metadata.metrics.default"] = "none"
+    fill_parquet_file_metadata(
+        datafile,
+        metadata,
+        len(file_bytes),
+        table_metadata,
+    )
+
+    assert len(datafile.value_counts) == 0
+    assert len(datafile.null_value_counts) == 0
+    assert len(datafile.nan_value_counts) == 0
+    assert len(datafile.lower_bounds) == 0
+    assert len(datafile.upper_bounds) == 0
+
+
+def test_metrics_mode_counts() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    table_metadata.properties["write.metadata.metrics.default"] = "counts"
+    fill_parquet_file_metadata(
+        datafile,
+        metadata,
+        len(file_bytes),
+        table_metadata,
+    )
+
+    assert len(datafile.value_counts) == 5
+    assert len(datafile.null_value_counts) == 5
+    assert len(datafile.nan_value_counts) == 0
+    assert len(datafile.lower_bounds) == 0
+    assert len(datafile.upper_bounds) == 0
+
+
+def test_metrics_mode_full() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    table_metadata.properties["write.metadata.metrics.default"] = "full"
+    fill_parquet_file_metadata(
+        datafile,
+        metadata,
+        len(file_bytes),
+        table_metadata,
+    )
+
+    assert len(datafile.value_counts) == 5
+    assert len(datafile.null_value_counts) == 5
+    assert len(datafile.nan_value_counts) == 0
+
+    assert len(datafile.lower_bounds) == 5
+    assert datafile.lower_bounds[1].decode() == "aaaaaaaaaaaaaaaaaaaa"
+    assert datafile.lower_bounds[2] == STRUCT_FLOAT.pack(1.69)
+    assert datafile.lower_bounds[5] == STRUCT_INT64.pack(1)
+    assert datafile.lower_bounds[6] == STRUCT_INT64.pack(1)
+    assert datafile.lower_bounds[7] == STRUCT_INT64.pack(2)
+
+    assert len(datafile.upper_bounds) == 5
+    assert datafile.upper_bounds[1].decode() == "zzzzzzzzzzzzzzzzzzzz"
+    assert datafile.upper_bounds[2] == STRUCT_FLOAT.pack(100)
+    assert datafile.upper_bounds[5] == STRUCT_INT64.pack(9)
+    assert datafile.upper_bounds[6] == STRUCT_INT64.pack(5)
+    assert datafile.upper_bounds[7] == STRUCT_INT64.pack(6)
+
+
+def test_metrics_mode_non_default_trunc() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    table_metadata.properties["write.metadata.metrics.default"] = "truncate(2)"
+    fill_parquet_file_metadata(
+        datafile,
+        metadata,
+        len(file_bytes),
+        table_metadata,
+    )
+
+    assert len(datafile.value_counts) == 5
+    assert len(datafile.null_value_counts) == 5
+    assert len(datafile.nan_value_counts) == 0
+
+    assert len(datafile.lower_bounds) == 5
+    assert datafile.lower_bounds[1].decode() == "aa"
+    assert datafile.lower_bounds[2] == STRUCT_FLOAT.pack(1.69)
+    assert datafile.lower_bounds[5] == STRUCT_INT64.pack(1)
+    assert datafile.lower_bounds[6] == STRUCT_INT64.pack(1)
+    assert datafile.lower_bounds[7] == STRUCT_INT64.pack(2)
+
+    assert len(datafile.upper_bounds) == 5
+    assert datafile.upper_bounds[1].decode() == "z{"
+    assert datafile.upper_bounds[2] == STRUCT_FLOAT.pack(100)
+    assert datafile.upper_bounds[5] == STRUCT_INT64.pack(9)
+    assert datafile.upper_bounds[6] == STRUCT_INT64.pack(5)
+    assert datafile.upper_bounds[7] == STRUCT_INT64.pack(6)
+
+
+def test_column_metrics_mode() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    table_metadata.properties["write.metadata.metrics.default"] = "truncate(2)"
+    table_metadata.properties["write.metadata.metrics.column.strings"] = "none"
+    table_metadata.properties["write.metadata.metrics.column.list.element"] = "counts"
+    fill_parquet_file_metadata(
+        datafile,
+        metadata,
+        len(file_bytes),
+        table_metadata,
+    )
+
+    assert len(datafile.value_counts) == 4
+    assert len(datafile.null_value_counts) == 4
+    assert len(datafile.nan_value_counts) == 0
+
+    assert len(datafile.lower_bounds) == 3
+    assert datafile.lower_bounds[2] == STRUCT_FLOAT.pack(1.69)
+    assert 5 not in datafile.lower_bounds

Review Comment:
   This doesn't look right. Integers and longs are not affected by truncate. For non-string or binary types, the mode should be equivalent to `full`.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1285259094


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1039,335 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.values()).union({"INT96"})
+
+
+class StatsAggregator:
+    current_min: Any
+    current_max: Any
+    trunc_length: Optional[int]
+
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min = None
+        self.current_max = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        expected_physical_type = _PRIMITIVE_TO_PHYSICAL[iceberg_type]
+        if expected_physical_type != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {expected_physical_type}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        self.current_min = val if self.current_min is None else min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        self.current_max = val if self.current_max is None else max(val, self.current_max)
+
+    def get_min(self) -> bytes:
+        return self.serialize(
+            self.current_min
+            if self.trunc_length is None
+            else TruncateTransform(width=self.trunc_length).transform(self.primitive_type)(self.current_min)
+        )
+
+    def get_max(self) -> Optional[bytes]:
+        if self.current_max is None:
+            return None
+
+        if self.primitive_type == StringType():
+            if type(self.current_max) != str:
+                raise ValueError("Expected the current_max to be a string")
+
+            s_result = self.current_max[: self.trunc_length]
+            if s_result != self.current_max:
+                chars = [*s_result]
+
+                for i in range(-1, -len(s_result) - 1, -1):
+                    try:
+                        to_inc = ord(chars[i])
+                        # will raise exception if the highest unicode code is reached
+                        _next = chr(to_inc + 1)
+                        chars[i] = _next
+                        return self.serialize("".join(chars))
+                    except ValueError:
+                        pass
+                return None  # didn't find a valid upper bound
+            return self.serialize(s_result)
+        elif self.primitive_type == BinaryType():
+            if type(self.current_max) != bytes:
+                raise ValueError("Expected the current_max to be bytes")
+            b_result = self.current_max[: self.trunc_length]
+            if b_result != self.current_max:
+                _bytes = [*b_result]
+                for i in range(-1, -len(b_result) - 1, -1):
+                    if _bytes[i] < 255:
+                        _bytes[i] += 1
+                        return b"".join([i.to_bytes(1, byteorder="little") for i in _bytes])
+                return None
+
+            return self.serialize(b_result)
+        else:
+            return self.serialize(self.current_max)[: self.trunc_length]
+
+
+DEFAULT_TRUNCATION_LENGTH = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+
+class MetricModeTypes(Enum):
+    TRUNCATE = "truncate"
+    NONE = "none"
+    COUNTS = "counts"
+    FULL = "full"
+
+
+DEFAULT_METRICS_MODE_KEY = "write.metadata.metrics.default"
+COLUMN_METRICS_MODE_KEY_PREFIX = "write.metadata.metrics.column"
+
+
+@dataclass(frozen=True)
+class MetricsMode(Singleton):
+    type: MetricModeTypes
+    length: Optional[int] = None
+
+
+def match_metrics_mode(mode: str) -> MetricsMode:
+    sanitized_mode = mode.lower()
+    if sanitized_mode.startswith("truncate"):
+        m = re.match(TRUNCATION_EXPR, mode, re.IGNORECASE)
+        if m:
+            length = int(m[1])
+            if length < 1:
+                raise ValueError("Truncation length must be larger than 0")
+            return MetricsMode(MetricModeTypes.TRUNCATE, int(m[1]))
+        else:
+            raise ValueError(f"Malformed truncate: {mode}")
+    elif sanitized_mode.startswith("none"):
+        return MetricsMode(MetricModeTypes.NONE)
+    elif sanitized_mode.startswith("counts"):
+        return MetricsMode(MetricModeTypes.COUNTS)
+    elif sanitized_mode.startswith("full"):
+        return MetricsMode(MetricModeTypes.FULL)
+    else:
+        raise ValueError(f"Unsupported metrics mode: {mode}")
+
+
+@dataclass(frozen=True)
+class StatisticsCollector:
+    field_id: int
+    iceberg_type: PrimitiveType
+    mode: MetricsMode
+    column_name: str
+
+
+class PyArrowStatisticsCollector(PreOrderSchemaVisitor[List[StatisticsCollector]]):
+    _field_id: int = 0
+    _schema: Schema
+    _properties: Dict[str, str]
+
+    def __init__(self, schema: Schema, properties: Dict[str, str]):
+        self._schema = schema
+        self._properties = properties
+
+    def schema(self, schema: Schema, struct_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        return struct_result()
+
+    def struct(
+        self, struct: StructType, field_results: List[Callable[[], List[StatisticsCollector]]]
+    ) -> List[StatisticsCollector]:
+        return list(chain(*[result() for result in field_results]))
+
+    def field(self, field: NestedField, field_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = field.field_id
+        result = field_result()
+        return result
+
+    def list(self, list_type: ListType, element_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = list_type.element_id
+        return element_result()
+
+    def map(
+        self,
+        map_type: MapType,
+        key_result: Callable[[], List[StatisticsCollector]],
+        value_result: Callable[[], List[StatisticsCollector]],
+    ) -> List[StatisticsCollector]:
+        self._field_id = map_type.key_id
+        k = key_result()
+        self._field_id = map_type.value_id
+        v = value_result()
+        return k + v
+
+    def primitive(self, primitive: PrimitiveType) -> List[StatisticsCollector]:
+        column_name = self._schema.find_column_name(self._field_id)
+        if column_name is None:
+            raise ValueError(f"Column for field {self._field_id} not found")

Review Comment:
   Why not just skip creating a stats collector for this field?



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1285256983


##########
python/pyiceberg/utils/file_stats.py:
##########
@@ -0,0 +1,164 @@
+#  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.
+
+from pyiceberg.manifest import DataFile, FileFormat
+import pyarrow.parquet as pq
+import pyarrow.compute as pc
+import pyarrow as pa
+import struct
+import datetime
+
+BOUND_TRUNCATED_LENGHT = 16
+
+# Serialization rules: https://iceberg.apache.org/spec/#binary-single-value-serialization
+#
+# Type      Binary serialization
+# boolean   0x00 for false, non-zero byte for true
+# int       Stored as 4-byte little-endian
+# long      Stored as 8-byte little-endian
+# float     Stored as 4-byte little-endian
+# double    Stored as 8-byte little-endian
+# date      Stores days from the 1970-01-01 in an 4-byte little-endian int
+# time      Stores microseconds from midnight in an 8-byte little-endian long
+# timestamp without zone	Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long
+# timestamp with zone	Stores microseconds from 1970-01-01 00:00:00.000000 UTC in an 8-byte little-endian long
+# string    UTF-8 bytes (without length)
+# uuid      16-byte big-endian value, see example in Appendix B
+# fixed(L)  Binary value
+# binary    Binary value (without length)
+#
+def serialize_to_binary(scalar: pa.Scalar) -> bytes:
+    value = scalar.as_py()
+    if isinstance(scalar, pa.BooleanScalar):
+        return struct.pack('?', value)
+    elif isinstance(scalar, (pa.Int8Scalar, pa.UInt8Scalar)):
+        return struct.pack('<b', value)
+    elif isinstance(scalar, (pa.Int16Scalar, pa.UInt16Scalar)):
+        return struct.pack('<h', value)
+    elif isinstance(scalar, (pa.Int32Scalar, pa.UInt32Scalar)):
+        return struct.pack('<i', value)
+    elif isinstance(scalar, (pa.Int64Scalar, pa.UInt64Scalar)):
+        return struct.pack('<q', value)
+    elif isinstance(scalar, pa.FloatScalar):
+        return struct.pack('<f', value)
+    elif isinstance(scalar, pa.DoubleScalar):
+        return struct.pack('<d', value)
+    elif isinstance(scalar, pa.StringScalar):
+        return value.encode('utf-8')
+    elif isinstance(scalar, pa.BinaryScalar):
+        return value
+    elif isinstance(scalar, (pa.Date32Scalar, pa.Date64Scalar)):
+        epoch = datetime.date(1970, 1, 1)
+        days = (value - epoch).days
+        return struct.pack('<i', days)
+    elif isinstance(scalar, (pa.Time32Scalar, pa.Time64Scalar)):
+        microseconds = int(value.hour * 60 * 60 * 1e6 +
+                        value.minute * 60 * 1e6 +
+                        value.second * 1e6 +
+                        value.microsecond)
+        return struct.pack('<q', microseconds)
+    elif isinstance(scalar, pa.TimestampScalar):
+        epoch = datetime.datetime(1970, 1, 1)
+        microseconds = int((value - epoch).total_seconds() * 1e6)
+        return struct.pack('<q', microseconds)
+    else:
+        raise TypeError('Unsupported type: {}'.format(type(scalar)))
+
+
+def fill_parquet_file_metadata(df: DataFile, file_object: pa.NativeFile, table: pa.Table = None) -> None:
+    """
+    Computes and fills the following fields of the DataFile object:
+
+    - file_format
+    - record_count
+    - file_size_in_bytes
+    - column_sizes
+    - value_counts
+    - null_value_counts
+    - nan_value_counts
+    - lower_bounds
+    - upper_bounds
+    - split_offsets
+    
+    Args:
+        df (DataFile): A DataFile object representing the Parquet file for which metadata is to be filled.
+        file_object (pa.NativeFile): A pyarrow NativeFile object pointing to the location where the 
+            Parquet file is stored.
+        table (pa.Table, optional): If the metadata is computed while writing a pyarrow Table to parquet
+            the table can be passed to compute the column statistics. If absent the table will be read
+            from file_object using pyarrow.parquet.read_table.
+    """
+    
+    parquet_file = pq.ParquetFile(file_object)

Review Comment:
   I completely agree here. We do not want to collect stats by reading individual data files. We should be able to get them when writing the file.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1285258347


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1039,335 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.values()).union({"INT96"})
+
+
+class StatsAggregator:
+    current_min: Any
+    current_max: Any
+    trunc_length: Optional[int]
+
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min = None
+        self.current_max = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        expected_physical_type = _PRIMITIVE_TO_PHYSICAL[iceberg_type]
+        if expected_physical_type != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {expected_physical_type}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        self.current_min = val if self.current_min is None else min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        self.current_max = val if self.current_max is None else max(val, self.current_max)
+
+    def get_min(self) -> bytes:
+        return self.serialize(
+            self.current_min
+            if self.trunc_length is None
+            else TruncateTransform(width=self.trunc_length).transform(self.primitive_type)(self.current_min)
+        )
+
+    def get_max(self) -> Optional[bytes]:
+        if self.current_max is None:
+            return None
+
+        if self.primitive_type == StringType():
+            if type(self.current_max) != str:
+                raise ValueError("Expected the current_max to be a string")
+
+            s_result = self.current_max[: self.trunc_length]
+            if s_result != self.current_max:
+                chars = [*s_result]
+
+                for i in range(-1, -len(s_result) - 1, -1):
+                    try:
+                        to_inc = ord(chars[i])
+                        # will raise exception if the highest unicode code is reached
+                        _next = chr(to_inc + 1)
+                        chars[i] = _next
+                        return self.serialize("".join(chars))
+                    except ValueError:
+                        pass
+                return None  # didn't find a valid upper bound
+            return self.serialize(s_result)
+        elif self.primitive_type == BinaryType():
+            if type(self.current_max) != bytes:
+                raise ValueError("Expected the current_max to be bytes")
+            b_result = self.current_max[: self.trunc_length]
+            if b_result != self.current_max:
+                _bytes = [*b_result]
+                for i in range(-1, -len(b_result) - 1, -1):
+                    if _bytes[i] < 255:
+                        _bytes[i] += 1
+                        return b"".join([i.to_bytes(1, byteorder="little") for i in _bytes])
+                return None
+
+            return self.serialize(b_result)
+        else:
+            return self.serialize(self.current_max)[: self.trunc_length]
+
+
+DEFAULT_TRUNCATION_LENGTH = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+
+class MetricModeTypes(Enum):
+    TRUNCATE = "truncate"
+    NONE = "none"
+    COUNTS = "counts"
+    FULL = "full"
+
+
+DEFAULT_METRICS_MODE_KEY = "write.metadata.metrics.default"
+COLUMN_METRICS_MODE_KEY_PREFIX = "write.metadata.metrics.column"
+
+
+@dataclass(frozen=True)
+class MetricsMode(Singleton):
+    type: MetricModeTypes
+    length: Optional[int] = None
+
+
+def match_metrics_mode(mode: str) -> MetricsMode:
+    sanitized_mode = mode.lower()
+    if sanitized_mode.startswith("truncate"):
+        m = re.match(TRUNCATION_EXPR, mode, re.IGNORECASE)

Review Comment:
   Why `re.IGNORECASE` when using `sanitized_mode` that has been converted to lower?



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] maxdebayser commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "maxdebayser (via GitHub)" <gi...@apache.org>.
maxdebayser commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1288916260


##########
python/tests/io/test_pyarrow.py:
##########
@@ -1345,3 +1374,655 @@ def test_pyarrow_wrap_fsspec(example_task: FileScanTask, table_schema_simple: Sc
 bar: [[1,2,3]]
 baz: [[true,false,null]]"""
     )
+
+
+def construct_test_table() -> Tuple[Any, Any, Union[TableMetadataV1, TableMetadataV2]]:
+    table_metadata = {
+        "format-version": 2,
+        "location": "s3://bucket/test/location",
+        "last-column-id": 7,
+        "current-schema-id": 0,
+        "schemas": [
+            {
+                "type": "struct",
+                "schema-id": 0,
+                "fields": [
+                    {"id": 1, "name": "strings", "required": False, "type": "string"},
+                    {"id": 2, "name": "floats", "required": False, "type": "float"},
+                    {
+                        "id": 3,
+                        "name": "list",
+                        "required": False,
+                        "type": {"type": "list", "element-id": 5, "element": "long", "element-required": False},
+                    },
+                    {
+                        "id": 4,
+                        "name": "maps",
+                        "required": False,
+                        "type": {
+                            "type": "map",
+                            "key-id": 6,
+                            "key": "long",
+                            "value-id": 7,
+                            "value": "long",
+                            "value-required": False,
+                        },
+                    },
+                ],
+            },
+        ],
+        "default-spec-id": 0,
+        "partition-specs": [{"spec-id": 0, "fields": []}],
+        "properties": {},
+    }
+
+    table_metadata = TableMetadataUtil.parse_obj(table_metadata)
+    arrow_schema = schema_to_pyarrow(table_metadata.schemas[0])
+
+    _strings = ["zzzzzzzzzzzzzzzzzzzz", "rrrrrrrrrrrrrrrrrrrr", None, "aaaaaaaaaaaaaaaaaaaa"]
+
+    _floats = [3.14, math.nan, 1.69, 100]
+
+    _list = [[1, 2, 3], [4, 5, 6], None, [7, 8, 9]]
+
+    _maps: List[Optional[Dict[int, int]]] = [
+        {1: 2, 3: 4},
+        None,
+        {5: 6},
+        {},
+    ]
+
+    table = pa.Table.from_pydict(
+        {
+            "strings": _strings,
+            "floats": _floats,
+            "list": _list,
+            "maps": _maps,
+        },
+        schema=arrow_schema,
+    )
+    metadata_collector: List[Any] = []
+
+    with pa.BufferOutputStream() as f:
+        with pq.ParquetWriter(f, table.schema, metadata_collector=metadata_collector) as writer:
+            writer.write_table(table)
+
+        return f.getvalue(), metadata_collector[0], table_metadata
+
+
+def test_record_count() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.record_count == 4
+
+
+def test_file_size() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.file_size_in_bytes == len(file_bytes)
+
+
+def test_value_counts() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.value_counts) == 5
+    assert datafile.value_counts[1] == 4
+    assert datafile.value_counts[2] == 4
+    assert datafile.value_counts[5] == 10  # 3 lists with 3 items and a None value
+    assert datafile.value_counts[6] == 5
+    assert datafile.value_counts[7] == 5
+
+
+def test_column_sizes() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.column_sizes) == 5
+    # these values are an artifact of how the write_table encodes the columns
+    assert datafile.column_sizes[1] == 116
+    assert datafile.column_sizes[2] == 89
+    assert datafile.column_sizes[5] == 151
+    assert datafile.column_sizes[6] == 117
+    assert datafile.column_sizes[7] == 117
+
+
+def test_null_and_nan_counts() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.null_value_counts) == 5
+    assert datafile.null_value_counts[1] == 1
+    assert datafile.null_value_counts[2] == 0
+    assert datafile.null_value_counts[5] == 1
+    assert datafile.null_value_counts[6] == 2
+    assert datafile.null_value_counts[7] == 2
+
+    # #arrow does not include this in the statistics
+    # assert len(datafile.nan_value_counts)  == 3
+    # assert datafile.nan_value_counts[1]    == 0

Review Comment:
   You're talking about the nan_value_counts, right? Pyarrow==12.0.1 doesn't compute this statistic unfortunately.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on pull request #7831: Python: Compute parquet stats

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#issuecomment-1727497615

   Thanks @rdblue for the review, that was very helpful. 


-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] maxdebayser commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "maxdebayser (via GitHub)" <gi...@apache.org>.
maxdebayser commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1288975288


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1039,335 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.values()).union({"INT96"})
+
+
+class StatsAggregator:
+    current_min: Any
+    current_max: Any
+    trunc_length: Optional[int]
+
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min = None
+        self.current_max = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        expected_physical_type = _PRIMITIVE_TO_PHYSICAL[iceberg_type]
+        if expected_physical_type != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {expected_physical_type}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        self.current_min = val if self.current_min is None else min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        self.current_max = val if self.current_max is None else max(val, self.current_max)
+
+    def get_min(self) -> bytes:
+        return self.serialize(
+            self.current_min
+            if self.trunc_length is None
+            else TruncateTransform(width=self.trunc_length).transform(self.primitive_type)(self.current_min)
+        )
+
+    def get_max(self) -> Optional[bytes]:
+        if self.current_max is None:
+            return None
+
+        if self.primitive_type == StringType():
+            if type(self.current_max) != str:
+                raise ValueError("Expected the current_max to be a string")
+
+            s_result = self.current_max[: self.trunc_length]
+            if s_result != self.current_max:
+                chars = [*s_result]
+
+                for i in range(-1, -len(s_result) - 1, -1):
+                    try:
+                        to_inc = ord(chars[i])
+                        # will raise exception if the highest unicode code is reached
+                        _next = chr(to_inc + 1)
+                        chars[i] = _next
+                        return self.serialize("".join(chars))
+                    except ValueError:
+                        pass
+                return None  # didn't find a valid upper bound
+            return self.serialize(s_result)
+        elif self.primitive_type == BinaryType():
+            if type(self.current_max) != bytes:
+                raise ValueError("Expected the current_max to be bytes")
+            b_result = self.current_max[: self.trunc_length]
+            if b_result != self.current_max:
+                _bytes = [*b_result]
+                for i in range(-1, -len(b_result) - 1, -1):
+                    if _bytes[i] < 255:
+                        _bytes[i] += 1
+                        return b"".join([i.to_bytes(1, byteorder="little") for i in _bytes])
+                return None
+
+            return self.serialize(b_result)
+        else:
+            return self.serialize(self.current_max)[: self.trunc_length]
+
+
+DEFAULT_TRUNCATION_LENGTH = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+
+class MetricModeTypes(Enum):
+    TRUNCATE = "truncate"
+    NONE = "none"
+    COUNTS = "counts"
+    FULL = "full"
+
+
+DEFAULT_METRICS_MODE_KEY = "write.metadata.metrics.default"
+COLUMN_METRICS_MODE_KEY_PREFIX = "write.metadata.metrics.column"
+
+
+@dataclass(frozen=True)
+class MetricsMode(Singleton):
+    type: MetricModeTypes
+    length: Optional[int] = None
+
+
+def match_metrics_mode(mode: str) -> MetricsMode:
+    sanitized_mode = mode.lower()
+    if sanitized_mode.startswith("truncate"):
+        m = re.match(TRUNCATION_EXPR, mode, re.IGNORECASE)
+        if m:
+            length = int(m[1])
+            if length < 1:
+                raise ValueError("Truncation length must be larger than 0")
+            return MetricsMode(MetricModeTypes.TRUNCATE, int(m[1]))
+        else:
+            raise ValueError(f"Malformed truncate: {mode}")
+    elif sanitized_mode.startswith("none"):
+        return MetricsMode(MetricModeTypes.NONE)
+    elif sanitized_mode.startswith("counts"):
+        return MetricsMode(MetricModeTypes.COUNTS)
+    elif sanitized_mode.startswith("full"):
+        return MetricsMode(MetricModeTypes.FULL)
+    else:
+        raise ValueError(f"Unsupported metrics mode: {mode}")
+
+
+@dataclass(frozen=True)
+class StatisticsCollector:
+    field_id: int
+    iceberg_type: PrimitiveType
+    mode: MetricsMode
+    column_name: str
+
+
+class PyArrowStatisticsCollector(PreOrderSchemaVisitor[List[StatisticsCollector]]):
+    _field_id: int = 0
+    _schema: Schema
+    _properties: Dict[str, str]
+
+    def __init__(self, schema: Schema, properties: Dict[str, str]):
+        self._schema = schema
+        self._properties = properties
+
+    def schema(self, schema: Schema, struct_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        return struct_result()
+
+    def struct(
+        self, struct: StructType, field_results: List[Callable[[], List[StatisticsCollector]]]
+    ) -> List[StatisticsCollector]:
+        return list(chain(*[result() for result in field_results]))
+
+    def field(self, field: NestedField, field_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = field.field_id
+        result = field_result()
+        return result
+
+    def list(self, list_type: ListType, element_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = list_type.element_id
+        return element_result()
+
+    def map(
+        self,
+        map_type: MapType,
+        key_result: Callable[[], List[StatisticsCollector]],
+        value_result: Callable[[], List[StatisticsCollector]],
+    ) -> List[StatisticsCollector]:
+        self._field_id = map_type.key_id
+        k = key_result()
+        self._field_id = map_type.value_id
+        v = value_result()
+        return k + v
+
+    def primitive(self, primitive: PrimitiveType) -> List[StatisticsCollector]:
+        column_name = self._schema.find_column_name(self._field_id)
+        if column_name is None:
+            raise ValueError(f"Column for field {self._field_id} not found")
+
+        metrics_mode = MetricsMode(MetricModeTypes.TRUNCATE, DEFAULT_TRUNCATION_LENGTH)
+
+        default_mode = self._properties.get(DEFAULT_METRICS_MODE_KEY)
+        if default_mode:
+            metrics_mode = match_metrics_mode(default_mode)
+
+        col_mode = self._properties.get(f"{COLUMN_METRICS_MODE_KEY_PREFIX}.{column_name}")
+        if col_mode:
+            metrics_mode = match_metrics_mode(col_mode)
+
+        if (
+            not (isinstance(primitive, StringType) or isinstance(primitive, BinaryType))
+            and metrics_mode.type == MetricModeTypes.TRUNCATE
+        ):
+            metrics_mode = MetricsMode(MetricModeTypes.FULL)
+
+        return [StatisticsCollector(field_id=self._field_id, iceberg_type=primitive, mode=metrics_mode, column_name=column_name)]
+
+
+def fill_parquet_file_metadata(
+    df: DataFile,
+    parquet_metadata: pq.FileMetaData,
+    file_size: int,
+    table_metadata: TableMetadata,
+) -> None:
+    """
+    Computes and fills the following fields of the DataFile object.
+
+    - file_format
+    - record_count
+    - file_size_in_bytes
+    - column_sizes
+    - value_counts
+    - null_value_counts
+    - nan_value_counts
+    - lower_bounds
+    - upper_bounds
+    - split_offsets
+
+    Args:
+        df (DataFile): A DataFile object representing the Parquet file for which metadata is to be filled.
+        parquet_metadata (pyarrow.parquet.FileMetaData): A pyarrow metadata object.
+        file_size (int): The total compressed file size cannot be retrieved from the metadata and hence has to
+            be passed here. Depending on the kind of file system and pyarrow library call used, different
+            ways to obtain this value might be appropriate.
+        table_metadata (pyiceberg.table.metadata.TableMetadata): The Iceberg table metadata. It is required to
+            compute the mapping if column position to iceberg schema type id. It's also used to set the mode
+            for column metrics collection
+    """
+    schema = next(filter(lambda s: s.schema_id == table_metadata.current_schema_id, table_metadata.schemas))
+
+    stats_columns = pre_order_visit(schema, PyArrowStatisticsCollector(schema, table_metadata.properties))
+
+    if parquet_metadata.num_columns != len(stats_columns):
+        raise ValueError(
+            f"Number of columns in metadata ({len(stats_columns)}) is different from the number of columns in pyarrow table ({parquet_metadata.num_columns})"
+        )
+
+    column_sizes: Dict[int, int] = {}
+    value_counts: Dict[int, int] = {}
+    split_offsets: List[int] = []
+
+    null_value_counts: Dict[int, int] = {}
+    nan_value_counts: Dict[int, int] = {}
+
+    col_aggs = {}
+
+    for r in range(parquet_metadata.num_row_groups):
+        # References:
+        # https://github.com/apache/iceberg/blob/fc381a81a1fdb8f51a0637ca27cd30673bd7aad3/parquet/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java#L232
+        # https://github.com/apache/parquet-mr/blob/ac29db4611f86a07cc6877b416aa4b183e09b353/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/ColumnChunkMetaData.java#L184
+
+        row_group = parquet_metadata.row_group(r)
+
+        data_offset = row_group.column(0).data_page_offset
+        dictionary_offset = row_group.column(0).dictionary_page_offset
+
+        if row_group.column(0).has_dictionary_page and dictionary_offset < data_offset:
+            split_offsets.append(dictionary_offset)
+        else:
+            split_offsets.append(data_offset)
+
+        for pos, stats_col in enumerate(stats_columns):
+            field_id = stats_col.field_id
+
+            column = row_group.column(pos)
+
+            column_sizes[field_id] = column_sizes.get(field_id, 0) + column.total_compressed_size
+
+            if stats_col.mode == MetricsMode(MetricModeTypes.NONE):
+                continue
+
+            value_counts[field_id] = value_counts.get(field_id, 0) + column.num_values
+
+            if column.is_stats_set:
+                try:
+                    statistics = column.statistics
+
+                    null_value_counts[field_id] = null_value_counts.get(field_id, 0) + statistics.null_count
+
+                    if stats_col.mode == MetricsMode(MetricModeTypes.COUNTS):
+                        continue
+
+                    if field_id not in col_aggs:
+                        col_aggs[field_id] = StatsAggregator(
+                            stats_col.iceberg_type, statistics.physical_type, stats_col.mode.length
+                        )
+
+                    col_aggs[field_id].add_min(statistics.min)
+                    col_aggs[field_id].add_max(statistics.max)
+
+                except pyarrow.lib.ArrowNotImplementedError as e:
+                    logger.warning(e)
+            else:
+                logger.warning("PyArrow statistics missing for column %d when writing file", pos)

Review Comment:
   Thanks. It's always better to protect ourselves against bugs in other libraries.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1289088051


##########
python/tests/io/test_pyarrow.py:
##########
@@ -1345,3 +1374,655 @@ def test_pyarrow_wrap_fsspec(example_task: FileScanTask, table_schema_simple: Sc
 bar: [[1,2,3]]
 baz: [[true,false,null]]"""
     )
+
+
+def construct_test_table() -> Tuple[Any, Any, Union[TableMetadataV1, TableMetadataV2]]:
+    table_metadata = {
+        "format-version": 2,
+        "location": "s3://bucket/test/location",
+        "last-column-id": 7,
+        "current-schema-id": 0,
+        "schemas": [
+            {
+                "type": "struct",
+                "schema-id": 0,
+                "fields": [
+                    {"id": 1, "name": "strings", "required": False, "type": "string"},
+                    {"id": 2, "name": "floats", "required": False, "type": "float"},
+                    {
+                        "id": 3,
+                        "name": "list",
+                        "required": False,
+                        "type": {"type": "list", "element-id": 5, "element": "long", "element-required": False},
+                    },
+                    {
+                        "id": 4,
+                        "name": "maps",
+                        "required": False,
+                        "type": {
+                            "type": "map",
+                            "key-id": 6,
+                            "key": "long",
+                            "value-id": 7,
+                            "value": "long",
+                            "value-required": False,
+                        },
+                    },
+                ],
+            },
+        ],
+        "default-spec-id": 0,
+        "partition-specs": [{"spec-id": 0, "fields": []}],
+        "properties": {},
+    }
+
+    table_metadata = TableMetadataUtil.parse_obj(table_metadata)
+    arrow_schema = schema_to_pyarrow(table_metadata.schemas[0])
+
+    _strings = ["zzzzzzzzzzzzzzzzzzzz", "rrrrrrrrrrrrrrrrrrrr", None, "aaaaaaaaaaaaaaaaaaaa"]
+
+    _floats = [3.14, math.nan, 1.69, 100]
+
+    _list = [[1, 2, 3], [4, 5, 6], None, [7, 8, 9]]
+
+    _maps: List[Optional[Dict[int, int]]] = [
+        {1: 2, 3: 4},
+        None,
+        {5: 6},
+        {},
+    ]
+
+    table = pa.Table.from_pydict(
+        {
+            "strings": _strings,
+            "floats": _floats,
+            "list": _list,
+            "maps": _maps,
+        },
+        schema=arrow_schema,
+    )
+    metadata_collector: List[Any] = []
+
+    with pa.BufferOutputStream() as f:
+        with pq.ParquetWriter(f, table.schema, metadata_collector=metadata_collector) as writer:
+            writer.write_table(table)
+
+        return f.getvalue(), metadata_collector[0], table_metadata
+
+
+def test_record_count() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.record_count == 4
+
+
+def test_file_size() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.file_size_in_bytes == len(file_bytes)
+
+
+def test_value_counts() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.value_counts) == 5
+    assert datafile.value_counts[1] == 4
+    assert datafile.value_counts[2] == 4
+    assert datafile.value_counts[5] == 10  # 3 lists with 3 items and a None value
+    assert datafile.value_counts[6] == 5

Review Comment:
   Spark has the same number of values:
   ![image](https://github.com/apache/iceberg/assets/1134248/ad3a7f71-2e7d-46af-8309-c2720b97ff75)
   



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1285262414


##########
python/tests/io/test_pyarrow.py:
##########
@@ -1345,3 +1374,655 @@ def test_pyarrow_wrap_fsspec(example_task: FileScanTask, table_schema_simple: Sc
 bar: [[1,2,3]]
 baz: [[true,false,null]]"""
     )
+
+
+def construct_test_table() -> Tuple[Any, Any, Union[TableMetadataV1, TableMetadataV2]]:
+    table_metadata = {
+        "format-version": 2,
+        "location": "s3://bucket/test/location",
+        "last-column-id": 7,
+        "current-schema-id": 0,
+        "schemas": [
+            {
+                "type": "struct",
+                "schema-id": 0,
+                "fields": [
+                    {"id": 1, "name": "strings", "required": False, "type": "string"},
+                    {"id": 2, "name": "floats", "required": False, "type": "float"},
+                    {
+                        "id": 3,
+                        "name": "list",
+                        "required": False,
+                        "type": {"type": "list", "element-id": 5, "element": "long", "element-required": False},
+                    },
+                    {
+                        "id": 4,
+                        "name": "maps",
+                        "required": False,
+                        "type": {
+                            "type": "map",
+                            "key-id": 6,
+                            "key": "long",
+                            "value-id": 7,
+                            "value": "long",
+                            "value-required": False,
+                        },
+                    },
+                ],
+            },
+        ],
+        "default-spec-id": 0,
+        "partition-specs": [{"spec-id": 0, "fields": []}],
+        "properties": {},
+    }
+
+    table_metadata = TableMetadataUtil.parse_obj(table_metadata)
+    arrow_schema = schema_to_pyarrow(table_metadata.schemas[0])
+
+    _strings = ["zzzzzzzzzzzzzzzzzzzz", "rrrrrrrrrrrrrrrrrrrr", None, "aaaaaaaaaaaaaaaaaaaa"]
+
+    _floats = [3.14, math.nan, 1.69, 100]
+
+    _list = [[1, 2, 3], [4, 5, 6], None, [7, 8, 9]]
+
+    _maps: List[Optional[Dict[int, int]]] = [
+        {1: 2, 3: 4},
+        None,
+        {5: 6},
+        {},
+    ]
+
+    table = pa.Table.from_pydict(

Review Comment:
   I think that this should have a nested struct without a list or map as well.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] maxdebayser commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "maxdebayser (via GitHub)" <gi...@apache.org>.
maxdebayser commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1267115691


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1013,3 +1025,271 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+class StatsAggregator:
+    def __init__(self, type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.trunc_length = trunc_length
+        self.primitive_type: Optional[PrimitiveType] = None
+
+        if type_string == "BOOLEAN":
+            self.primitive_type = BooleanType()
+        elif type_string == "INT32":
+            self.primitive_type = IntegerType()
+        elif type_string == "INT64":
+            self.primitive_type = LongType()
+        elif type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+        elif type_string == "FLOAT":
+            self.primitive_type = FloatType()
+        elif type_string == "DOUBLE":
+            self.primitive_type = DoubleType()
+        elif type_string == "BYTE_ARRAY":
+            self.primitive_type = BinaryType()
+        elif type_string == "FIXED_LEN_BYTE_ARRAY":
+            self.primitive_type = BinaryType()
+        else:
+            raise AssertionError(f"Unknown physical type {type_string}")

Review Comment:
   I agree that the code can be simplified like that. I just want to run a thought by you: since we have this redundancy in type information should we perhaps take advantage of this to check that the types match? In this way we would be able to detect bugs that result in the data written to parquet being of a different type than what is in the schema.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] maxdebayser commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "maxdebayser (via GitHub)" <gi...@apache.org>.
maxdebayser commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1281068408


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1040,291 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHISICAL_TYPES = ["BOOLEAN", "INT32", "INT64", "INT96", "FLOAT", "DOUBLE", "BYTE_ARRAY", "FIXED_LEN_BYTE_ARRAY"]
+
+
+class StatsAggregator:
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.trunc_length = trunc_length
+
+        assert physical_type_string in _PHISICAL_TYPES, f"Unknown physical type {physical_type_string}"
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+        assert (
+            _PRIMITIVE_TO_PHYSICAL[iceberg_type] == physical_type_string
+        ), f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {_PRIMITIVE_TO_PHYSICAL[iceberg_type]}"
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if type(value) == date:

Review Comment:
   What I meant was can we merge it like this and then refactor in the other 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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] maxdebayser commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "maxdebayser (via GitHub)" <gi...@apache.org>.
maxdebayser commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1280674118


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1040,291 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHISICAL_TYPES = ["BOOLEAN", "INT32", "INT64", "INT96", "FLOAT", "DOUBLE", "BYTE_ARRAY", "FIXED_LEN_BYTE_ARRAY"]
+
+
+class StatsAggregator:
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.trunc_length = trunc_length
+
+        assert physical_type_string in _PHISICAL_TYPES, f"Unknown physical type {physical_type_string}"
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+        assert (
+            _PRIMITIVE_TO_PHYSICAL[iceberg_type] == physical_type_string
+        ), f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {_PRIMITIVE_TO_PHYSICAL[iceberg_type]}"
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if type(value) == date:

Review Comment:
   Since this commit already has test cases that depend on this conversion, can we address the refactoring in that 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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1285259229


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1039,335 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.values()).union({"INT96"})
+
+
+class StatsAggregator:
+    current_min: Any
+    current_max: Any
+    trunc_length: Optional[int]
+
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min = None
+        self.current_max = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        expected_physical_type = _PRIMITIVE_TO_PHYSICAL[iceberg_type]
+        if expected_physical_type != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {expected_physical_type}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        self.current_min = val if self.current_min is None else min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        self.current_max = val if self.current_max is None else max(val, self.current_max)
+
+    def get_min(self) -> bytes:
+        return self.serialize(
+            self.current_min
+            if self.trunc_length is None
+            else TruncateTransform(width=self.trunc_length).transform(self.primitive_type)(self.current_min)
+        )
+
+    def get_max(self) -> Optional[bytes]:
+        if self.current_max is None:
+            return None
+
+        if self.primitive_type == StringType():
+            if type(self.current_max) != str:
+                raise ValueError("Expected the current_max to be a string")
+
+            s_result = self.current_max[: self.trunc_length]
+            if s_result != self.current_max:
+                chars = [*s_result]
+
+                for i in range(-1, -len(s_result) - 1, -1):
+                    try:
+                        to_inc = ord(chars[i])
+                        # will raise exception if the highest unicode code is reached
+                        _next = chr(to_inc + 1)
+                        chars[i] = _next
+                        return self.serialize("".join(chars))
+                    except ValueError:
+                        pass
+                return None  # didn't find a valid upper bound
+            return self.serialize(s_result)
+        elif self.primitive_type == BinaryType():
+            if type(self.current_max) != bytes:
+                raise ValueError("Expected the current_max to be bytes")
+            b_result = self.current_max[: self.trunc_length]
+            if b_result != self.current_max:
+                _bytes = [*b_result]
+                for i in range(-1, -len(b_result) - 1, -1):
+                    if _bytes[i] < 255:
+                        _bytes[i] += 1
+                        return b"".join([i.to_bytes(1, byteorder="little") for i in _bytes])
+                return None
+
+            return self.serialize(b_result)
+        else:
+            return self.serialize(self.current_max)[: self.trunc_length]
+
+
+DEFAULT_TRUNCATION_LENGTH = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+
+class MetricModeTypes(Enum):
+    TRUNCATE = "truncate"
+    NONE = "none"
+    COUNTS = "counts"
+    FULL = "full"
+
+
+DEFAULT_METRICS_MODE_KEY = "write.metadata.metrics.default"
+COLUMN_METRICS_MODE_KEY_PREFIX = "write.metadata.metrics.column"
+
+
+@dataclass(frozen=True)
+class MetricsMode(Singleton):
+    type: MetricModeTypes
+    length: Optional[int] = None
+
+
+def match_metrics_mode(mode: str) -> MetricsMode:
+    sanitized_mode = mode.lower()
+    if sanitized_mode.startswith("truncate"):
+        m = re.match(TRUNCATION_EXPR, mode, re.IGNORECASE)
+        if m:
+            length = int(m[1])
+            if length < 1:
+                raise ValueError("Truncation length must be larger than 0")
+            return MetricsMode(MetricModeTypes.TRUNCATE, int(m[1]))
+        else:
+            raise ValueError(f"Malformed truncate: {mode}")
+    elif sanitized_mode.startswith("none"):
+        return MetricsMode(MetricModeTypes.NONE)
+    elif sanitized_mode.startswith("counts"):
+        return MetricsMode(MetricModeTypes.COUNTS)
+    elif sanitized_mode.startswith("full"):
+        return MetricsMode(MetricModeTypes.FULL)
+    else:
+        raise ValueError(f"Unsupported metrics mode: {mode}")
+
+
+@dataclass(frozen=True)
+class StatisticsCollector:
+    field_id: int
+    iceberg_type: PrimitiveType
+    mode: MetricsMode
+    column_name: str
+
+
+class PyArrowStatisticsCollector(PreOrderSchemaVisitor[List[StatisticsCollector]]):
+    _field_id: int = 0
+    _schema: Schema
+    _properties: Dict[str, str]
+
+    def __init__(self, schema: Schema, properties: Dict[str, str]):
+        self._schema = schema
+        self._properties = properties
+
+    def schema(self, schema: Schema, struct_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        return struct_result()
+
+    def struct(
+        self, struct: StructType, field_results: List[Callable[[], List[StatisticsCollector]]]
+    ) -> List[StatisticsCollector]:
+        return list(chain(*[result() for result in field_results]))
+
+    def field(self, field: NestedField, field_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = field.field_id
+        result = field_result()
+        return result
+
+    def list(self, list_type: ListType, element_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = list_type.element_id
+        return element_result()
+
+    def map(
+        self,
+        map_type: MapType,
+        key_result: Callable[[], List[StatisticsCollector]],
+        value_result: Callable[[], List[StatisticsCollector]],
+    ) -> List[StatisticsCollector]:
+        self._field_id = map_type.key_id
+        k = key_result()
+        self._field_id = map_type.value_id
+        v = value_result()
+        return k + v
+
+    def primitive(self, primitive: PrimitiveType) -> List[StatisticsCollector]:
+        column_name = self._schema.find_column_name(self._field_id)
+        if column_name is None:
+            raise ValueError(f"Column for field {self._field_id} not found")
+
+        metrics_mode = MetricsMode(MetricModeTypes.TRUNCATE, DEFAULT_TRUNCATION_LENGTH)
+
+        default_mode = self._properties.get(DEFAULT_METRICS_MODE_KEY)
+        if default_mode:
+            metrics_mode = match_metrics_mode(default_mode)

Review Comment:
   The default mode construction can be done in the constructor rather than in every leaf node, right?



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1285258279


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1039,335 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.values()).union({"INT96"})
+
+
+class StatsAggregator:
+    current_min: Any
+    current_max: Any
+    trunc_length: Optional[int]
+
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min = None
+        self.current_max = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        expected_physical_type = _PRIMITIVE_TO_PHYSICAL[iceberg_type]
+        if expected_physical_type != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {expected_physical_type}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        self.current_min = val if self.current_min is None else min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        self.current_max = val if self.current_max is None else max(val, self.current_max)
+
+    def get_min(self) -> bytes:
+        return self.serialize(
+            self.current_min
+            if self.trunc_length is None
+            else TruncateTransform(width=self.trunc_length).transform(self.primitive_type)(self.current_min)
+        )
+
+    def get_max(self) -> Optional[bytes]:
+        if self.current_max is None:
+            return None
+
+        if self.primitive_type == StringType():
+            if type(self.current_max) != str:
+                raise ValueError("Expected the current_max to be a string")
+
+            s_result = self.current_max[: self.trunc_length]
+            if s_result != self.current_max:
+                chars = [*s_result]
+
+                for i in range(-1, -len(s_result) - 1, -1):
+                    try:
+                        to_inc = ord(chars[i])
+                        # will raise exception if the highest unicode code is reached
+                        _next = chr(to_inc + 1)
+                        chars[i] = _next
+                        return self.serialize("".join(chars))
+                    except ValueError:
+                        pass
+                return None  # didn't find a valid upper bound
+            return self.serialize(s_result)
+        elif self.primitive_type == BinaryType():
+            if type(self.current_max) != bytes:
+                raise ValueError("Expected the current_max to be bytes")
+            b_result = self.current_max[: self.trunc_length]
+            if b_result != self.current_max:
+                _bytes = [*b_result]
+                for i in range(-1, -len(b_result) - 1, -1):
+                    if _bytes[i] < 255:
+                        _bytes[i] += 1
+                        return b"".join([i.to_bytes(1, byteorder="little") for i in _bytes])
+                return None
+
+            return self.serialize(b_result)
+        else:
+            return self.serialize(self.current_max)[: self.trunc_length]
+
+
+DEFAULT_TRUNCATION_LENGTH = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"

Review Comment:
   Since this is strict with string start and end, we should make sure that `trim` is called to remove accidental whitespace.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] maxdebayser commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "maxdebayser (via GitHub)" <gi...@apache.org>.
maxdebayser commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1258846345


##########
python/pyiceberg/utils/file_stats.py:
##########
@@ -0,0 +1,333 @@
+#  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.
+
+import struct
+from typing import (
+    Any,
+    Dict,
+    List,
+    Union,
+)
+
+import pyarrow.lib
+import pyarrow.parquet as pq
+
+from pyiceberg.manifest import DataFile, FileFormat
+from pyiceberg.schema import Schema, SchemaVisitor, visit
+from pyiceberg.types import (
+    IcebergType,
+    ListType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StructType,
+)
+
+BOUND_TRUNCATED_LENGHT = 16
+
+# Serialization rules: https://iceberg.apache.org/spec/#binary-single-value-serialization
+#
+# Type      Binary serialization
+# boolean   0x00 for false, non-zero byte for true
+# int       Stored as 4-byte little-endian
+# long      Stored as 8-byte little-endian
+# float     Stored as 4-byte little-endian
+# double    Stored as 8-byte little-endian
+# date      Stores days from the 1970-01-01 in an 4-byte little-endian int
+# time      Stores microseconds from midnight in an 8-byte little-endian long
+# timestamp without zone	Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long
+# timestamp with zone	Stores microseconds from 1970-01-01 00:00:00.000000 UTC in an 8-byte little-endian long
+# string    UTF-8 bytes (without length)
+# uuid      16-byte big-endian value, see example in Appendix B
+# fixed(L)  Binary value
+# binary    Binary value (without length)
+#
+
+
+def bool_to_avro(value: bool) -> bytes:
+    return struct.pack("?", value)
+
+
+def int32_to_avro(value: int) -> bytes:
+    return struct.pack("<i", value)
+
+
+def int64_to_avro(value: int) -> bytes:
+    return struct.pack("<q", value)
+
+
+def float_to_avro(value: float) -> bytes:
+    return struct.pack("<f", value)
+
+
+def double_to_avro(value: float) -> bytes:
+    return struct.pack("<d", value)
+
+
+def bytes_to_avro(value: Union[bytes, str]) -> bytes:
+    if type(value) == str:
+        return value.encode()
+    else:
+        assert isinstance(value, bytes)  # appeases mypy
+        return value
+
+
+class StatsAggregator:
+    def __init__(self, type_string: str):
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.serialize: Any = None
+
+        if type_string == "BOOLEAN":
+            self.serialize = bool_to_avro
+        elif type_string == "INT32":
+            self.serialize = int32_to_avro
+        elif type_string == "INT64":
+            self.serialize = int64_to_avro
+        elif type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+        elif type_string == "FLOAT":
+            self.serialize = float_to_avro
+        elif type_string == "DOUBLE":
+            self.serialize = double_to_avro
+        elif type_string == "BYTE_ARRAY":
+            self.serialize = bytes_to_avro
+        elif type_string == "FIXED_LEN_BYTE_ARRAY":
+            self.serialize = bytes_to_avro
+        else:
+            raise AssertionError(f"Unknown physical type {type_string}")
+
+    def add_min(self, val: bytes) -> None:
+        if not self.current_min:
+            self.current_min = val
+        elif val < self.current_min:
+            self.current_min = val
+
+    def add_max(self, val: bytes) -> None:
+        if not self.current_max:
+            self.current_max = val
+        elif self.current_max < val:
+            self.current_max = val
+
+    def get_min(self) -> bytes:
+        return self.serialize(self.current_min)[:BOUND_TRUNCATED_LENGHT]
+
+    def get_max(self) -> bytes:
+        return self.serialize(self.current_max)[:BOUND_TRUNCATED_LENGHT]
+
+
+def fill_parquet_file_metadata(
+    df: DataFile, metadata: pq.FileMetaData, col_path_2_iceberg_id: Dict[str, int], file_size: int
+) -> None:
+    """
+    Computes and fills the following fields of the DataFile object.
+
+    - file_format
+    - record_count
+    - file_size_in_bytes
+    - column_sizes
+    - value_counts
+    - null_value_counts
+    - nan_value_counts
+    - lower_bounds
+    - upper_bounds
+    - split_offsets
+
+    Args:
+        df (DataFile): A DataFile object representing the Parquet file for which metadata is to be filled.
+        metadata (pyarrow.parquet.FileMetaData): A pyarrow metadata object.
+        col_path_2_iceberg_id: A mapping of column paths as in the `path_in_schema` attribute of the colum
+            metadata to iceberg schema IDs. For scalar columns this will be the column name. For complex types
+            it could be something like `my_map.key_value.value`
+        file_size (int): The total compressed file size cannot be retrieved from the metadata and hence has to
+            be passed here. Depending on the kind of file system and pyarrow library call used, different
+            ways to obtain this value might be appropriate.
+    """
+    col_index_2_id = {}
+
+    col_names = set(metadata.schema.names)
+
+    first_group = metadata.row_group(0)
+
+    for c in range(metadata.num_columns):
+        column = first_group.column(c)
+        col_path = column.path_in_schema
+
+        if col_path in col_path_2_iceberg_id:
+            col_index_2_id[c] = col_path_2_iceberg_id[col_path]
+        else:
+            raise AssertionError(f"Column path {col_path} couldn't be mapped to an iceberg ID")
+
+    column_sizes: Dict[int, int] = {}
+    value_counts: Dict[int, int] = {}
+    split_offsets: List[int] = []
+
+    null_value_counts: Dict[int, int] = {}
+    nan_value_counts: Dict[int, int] = {}
+
+    col_aggs = {}
+
+    for r in range(metadata.num_row_groups):
+        # References:
+        # https://github.com/apache/iceberg/blob/fc381a81a1fdb8f51a0637ca27cd30673bd7aad3/parquet/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java#L232
+        # https://github.com/apache/parquet-mr/blob/ac29db4611f86a07cc6877b416aa4b183e09b353/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/ColumnChunkMetaData.java#L184
+
+        row_group = metadata.row_group(r)
+
+        data_offset = row_group.column(0).data_page_offset
+        dictionary_offset = row_group.column(0).dictionary_page_offset
+
+        if row_group.column(0).has_dictionary_page and dictionary_offset < data_offset:
+            split_offsets.append(dictionary_offset)
+        else:
+            split_offsets.append(data_offset)
+
+        for c in range(metadata.num_columns):
+            col_id = col_index_2_id[c]
+
+            column = row_group.column(c)
+
+            column_sizes[col_id] = column_sizes.get(col_id, 0) + column.total_compressed_size
+            value_counts[col_id] = value_counts.get(col_id, 0) + column.num_values
+
+            if column.is_stats_set:
+                try:
+                    statistics = column.statistics
+
+                    null_value_counts[col_id] = null_value_counts.get(col_id, 0) + statistics.null_count
+
+                    if column.path_in_schema in col_names:
+                        # Iceberg seems to only have statistics for scalar columns
+
+                        if col_id not in col_aggs:
+                            col_aggs[col_id] = StatsAggregator(statistics.physical_type)
+
+                        col_aggs[col_id].add_min(statistics.min)

Review Comment:
   It's because there are 3 different concerns here:
   - dealing with the parquet type that leaks through the arrow API: https://github.com/apache/arrow/blob/d676078c13a02ad920eeea2acd5fa517f14526e2/cpp/src/parquet/parquet.thrift#L34
   - dealing with with the metrics mode (full or truncate)
   - actually computing min and max.
   
   I think these should stay out of the inner loop to keep it readable.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1259667233


##########
python/tests/io/test_pyarrow.py:
##########
@@ -1330,3 +1341,485 @@ def test_pyarrow_wrap_fsspec(example_task: FileScanTask, table_schema_simple: Sc
 bar: [[1,2,3]]
 baz: [[true,false,null]]"""
     )
+
+
+def construct_test_table() -> pa.Buffer:
+    schema = pa.schema(
+        [
+            pa.field("strings", pa.string()),
+            pa.field("floats", pa.float64()),
+            pa.field("list", pa.list_(pa.int64())),
+            pa.field("maps", pa.map_(pa.int64(), pa.int64())),
+        ]
+    )
+
+    _strings = ["zzzzzzzzzzzzzzzzzzzz", "rrrrrrrrrrrrrrrrrrrr", None, "aaaaaaaaaaaaaaaaaaaa"]
+
+    _floats = [3.14, math.nan, 1.69, 100]
+
+    _list = [[1, 2, 3], [4, 5, 6], None, [7, 8, 9]]
+
+    _maps: List[Optional[Dict[int, int]]] = [
+        {1: 2, 3: 4},
+        None,
+        {5: 6},
+        {},
+    ]
+
+    table = pa.Table.from_pydict(
+        {
+            "strings": _strings,
+            "floats": _floats,
+            "list": _list,
+            "maps": _maps,
+        },
+        schema=schema,
+    )
+    f = pa.BufferOutputStream()
+
+    metadata_collector: List[Any] = []
+    writer = pq.ParquetWriter(f, table.schema, metadata_collector=metadata_collector)
+
+    writer.write_table(table)
+    writer.close()
+
+    print(writer.writer)
+    print(writer.writer.metadata)
+
+    mapping = {"strings": 1, "floats": 2, "list.list.item": 3, "maps.key_value.key": 4, "maps.key_value.value": 5}
+
+    return f.getvalue(), metadata_collector[0], mapping
+
+
+def test_record_count() -> None:
+    (file_bytes, metadata, mapping) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, mapping, len(file_bytes))
+
+    assert datafile.record_count == 4
+
+
+def test_file_size() -> None:
+    (file_bytes, metadata, mapping) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, mapping, len(file_bytes))
+
+    assert datafile.file_size_in_bytes == len(file_bytes)
+
+
+def test_value_counts() -> None:
+    (file_bytes, metadata, mapping) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, mapping, len(file_bytes))
+
+    assert len(datafile.value_counts) == 5
+    assert datafile.value_counts[1] == 4
+    assert datafile.value_counts[2] == 4
+    assert datafile.value_counts[3] == 10  # 3 lists with 3 items and a None value
+    assert datafile.value_counts[4] == 5
+    assert datafile.value_counts[5] == 5
+
+
+def test_column_sizes() -> None:
+    (file_bytes, metadata, mapping) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, mapping, len(file_bytes))
+
+    assert len(datafile.column_sizes) == 5
+    # these values are an artifact of how the write_table encodes the columns
+    assert datafile.column_sizes[1] == 116
+    assert datafile.column_sizes[2] == 119
+    assert datafile.column_sizes[3] == 151
+    assert datafile.column_sizes[4] == 117
+    assert datafile.column_sizes[5] == 117
+
+
+def test_null_and_nan_counts() -> None:
+    (file_bytes, metadata, mapping) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, mapping, len(file_bytes))
+
+    assert len(datafile.null_value_counts) == 5
+    assert datafile.null_value_counts[1] == 1
+    assert datafile.null_value_counts[2] == 0
+    assert datafile.null_value_counts[3] == 1
+    assert datafile.null_value_counts[4] == 2
+    assert datafile.null_value_counts[5] == 2
+
+    # #arrow does not include this in the statistics
+    # assert len(datafile.nan_value_counts)  == 3
+    # assert datafile.nan_value_counts[1]    == 0
+    # assert datafile.nan_value_counts[2]    == 1
+    # assert datafile.nan_value_counts[3]    == 0
+
+
+def test_bounds() -> None:
+    (file_bytes, metadata, mapping) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, mapping, len(file_bytes))
+
+    assert len(datafile.lower_bounds) == 2
+    assert datafile.lower_bounds[1].decode() == "aaaaaaaaaaaaaaaaaaaa"[:BOUND_TRUNCATED_LENGHT]
+    assert datafile.lower_bounds[2] == STRUCT_DOUBLE.pack(1.69)
+
+    assert len(datafile.upper_bounds) == 2
+    assert datafile.upper_bounds[1].decode() == "zzzzzzzzzzzzzzzzzzzz"[:BOUND_TRUNCATED_LENGHT]
+    assert datafile.upper_bounds[2] == STRUCT_DOUBLE.pack(100)
+
+
+def test_metrics_mode_none(example_table_metadata_v2: Dict[str, Any]) -> None:
+    (file_bytes, metadata, mapping) = construct_test_table()
+
+    datafile = DataFile()
+    table_metadata = TableMetadataUtil.parse_obj(example_table_metadata_v2)
+    table_metadata.properties["write.metadata.metrics.default"] = "none"
+    fill_parquet_file_metadata(
+        datafile,
+        metadata,
+        mapping,
+        len(file_bytes),
+        table_metadata,
+    )
+
+    assert len(datafile.value_counts) == 0
+    assert len(datafile.null_value_counts) == 0
+    assert len(datafile.nan_value_counts) == 0
+    assert len(datafile.lower_bounds) == 0
+    assert len(datafile.upper_bounds) == 0
+
+
+def test_metrics_mode_counts(example_table_metadata_v2: Dict[str, Any]) -> None:
+    (file_bytes, metadata, mapping) = construct_test_table()
+
+    datafile = DataFile()
+    table_metadata = TableMetadataUtil.parse_obj(example_table_metadata_v2)
+    table_metadata.properties["write.metadata.metrics.default"] = "counts"
+    fill_parquet_file_metadata(
+        datafile,
+        metadata,
+        mapping,
+        len(file_bytes),
+        table_metadata,
+    )
+
+    assert len(datafile.value_counts) == 5
+    assert len(datafile.null_value_counts) == 5
+    assert len(datafile.nan_value_counts) == 0
+    assert len(datafile.lower_bounds) == 0
+    assert len(datafile.upper_bounds) == 0
+
+
+def test_metrics_mode_full(example_table_metadata_v2: Dict[str, Any]) -> None:
+    (file_bytes, metadata, mapping) = construct_test_table()
+
+    datafile = DataFile()
+    table_metadata = TableMetadataUtil.parse_obj(example_table_metadata_v2)
+    table_metadata.properties["write.metadata.metrics.default"] = "full"
+    fill_parquet_file_metadata(
+        datafile,
+        metadata,
+        mapping,
+        len(file_bytes),
+        table_metadata,
+    )
+
+    assert len(datafile.value_counts) == 5
+    assert len(datafile.null_value_counts) == 5
+    assert len(datafile.nan_value_counts) == 0
+
+    assert len(datafile.lower_bounds) == 2
+    assert datafile.lower_bounds[1].decode() == "aaaaaaaaaaaaaaaaaaaa"
+    assert datafile.lower_bounds[2] == STRUCT_DOUBLE.pack(1.69)
+
+    assert len(datafile.upper_bounds) == 2
+    assert datafile.upper_bounds[1].decode() == "zzzzzzzzzzzzzzzzzzzz"
+    assert datafile.upper_bounds[2] == STRUCT_DOUBLE.pack(100)
+
+
+def test_metrics_mode_non_default_trunc(example_table_metadata_v2: Dict[str, Any]) -> None:
+    (file_bytes, metadata, mapping) = construct_test_table()
+
+    datafile = DataFile()
+    table_metadata = TableMetadataUtil.parse_obj(example_table_metadata_v2)
+    table_metadata.properties["write.metadata.metrics.default"] = "truncate(2)"
+    fill_parquet_file_metadata(
+        datafile,
+        metadata,
+        mapping,
+        len(file_bytes),
+        table_metadata,
+    )
+
+    assert len(datafile.value_counts) == 5
+    assert len(datafile.null_value_counts) == 5
+    assert len(datafile.nan_value_counts) == 0
+
+    assert len(datafile.lower_bounds) == 2
+    assert datafile.lower_bounds[1].decode() == "aa"
+    assert datafile.lower_bounds[2] == STRUCT_DOUBLE.pack(1.69)[:2]
+
+    assert len(datafile.upper_bounds) == 2
+    assert datafile.upper_bounds[1].decode() == "zz"
+    assert datafile.upper_bounds[2] == STRUCT_DOUBLE.pack(100)[:2]
+
+
+def test_column_metrics_mode(example_table_metadata_v2: Dict[str, Any]) -> None:
+    (file_bytes, metadata, mapping) = construct_test_table()
+
+    datafile = DataFile()
+    table_metadata = TableMetadataUtil.parse_obj(example_table_metadata_v2)
+    table_metadata.properties["write.metadata.metrics.default"] = "truncate(2)"
+    table_metadata.properties["write.metadata.metrics.column.strings"] = "none"
+    fill_parquet_file_metadata(
+        datafile,
+        metadata,
+        mapping,
+        len(file_bytes),
+        table_metadata,
+    )
+
+    assert len(datafile.value_counts) == 4
+    assert len(datafile.null_value_counts) == 4
+    assert len(datafile.nan_value_counts) == 0
+
+    assert len(datafile.lower_bounds) == 1
+    assert datafile.lower_bounds[2] == STRUCT_DOUBLE.pack(1.69)[:2]
+
+    assert len(datafile.upper_bounds) == 1
+    assert datafile.upper_bounds[2] == STRUCT_DOUBLE.pack(100)[:2]
+
+
+def test_offsets() -> None:
+    (file_bytes, metadata, mapping) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, mapping, len(file_bytes))
+
+    assert datafile.split_offsets is not None
+    assert len(datafile.split_offsets) == 1
+    assert datafile.split_offsets[0] == 4
+
+
+def test_dataset() -> pa.Buffer:
+    schema = pa.schema([pa.field("ints", pa.int64()), pa.field("even", pa.bool_())])
+
+    _ints = [0, 2, 4, 8, 1, 3, 5, 7]
+    parity = [True, True, True, True, False, False, False, False]
+
+    table = pa.Table.from_pydict({"ints": _ints, "even": parity}, schema=schema)
+
+    visited_paths = []
+
+    def file_visitor(written_file: Any) -> None:
+        visited_paths.append(written_file)
+
+    with TemporaryDirectory() as tmpdir:
+        pq.write_to_dataset(table, tmpdir, partition_cols=["even"], file_visitor=file_visitor)

Review Comment:
   Let's leave out partitioning for now. We first have to investigate how to handle applying the partition transforms.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] maxdebayser commented on pull request #7831: Python: Compute parquet stats

Posted by "maxdebayser (via GitHub)" <gi...@apache.org>.
maxdebayser commented on PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#issuecomment-1727691008

   Thanks, @Fokko and @rdblue 


-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1259770703


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1013,3 +1027,359 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+BOUND_TRUNCATED_LENGHT = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"

Review Comment:
   Oh, that's a great catch, missed that one. Very sad :'(



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1285258195


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1039,335 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.values()).union({"INT96"})
+
+
+class StatsAggregator:
+    current_min: Any
+    current_max: Any
+    trunc_length: Optional[int]
+
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min = None
+        self.current_max = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        expected_physical_type = _PRIMITIVE_TO_PHYSICAL[iceberg_type]
+        if expected_physical_type != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {expected_physical_type}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        self.current_min = val if self.current_min is None else min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        self.current_max = val if self.current_max is None else max(val, self.current_max)
+
+    def get_min(self) -> bytes:
+        return self.serialize(
+            self.current_min
+            if self.trunc_length is None
+            else TruncateTransform(width=self.trunc_length).transform(self.primitive_type)(self.current_min)
+        )
+
+    def get_max(self) -> Optional[bytes]:
+        if self.current_max is None:
+            return None
+
+        if self.primitive_type == StringType():
+            if type(self.current_max) != str:
+                raise ValueError("Expected the current_max to be a string")
+
+            s_result = self.current_max[: self.trunc_length]
+            if s_result != self.current_max:
+                chars = [*s_result]
+
+                for i in range(-1, -len(s_result) - 1, -1):
+                    try:
+                        to_inc = ord(chars[i])
+                        # will raise exception if the highest unicode code is reached
+                        _next = chr(to_inc + 1)
+                        chars[i] = _next
+                        return self.serialize("".join(chars))
+                    except ValueError:
+                        pass
+                return None  # didn't find a valid upper bound
+            return self.serialize(s_result)
+        elif self.primitive_type == BinaryType():
+            if type(self.current_max) != bytes:
+                raise ValueError("Expected the current_max to be bytes")
+            b_result = self.current_max[: self.trunc_length]
+            if b_result != self.current_max:
+                _bytes = [*b_result]
+                for i in range(-1, -len(b_result) - 1, -1):
+                    if _bytes[i] < 255:
+                        _bytes[i] += 1
+                        return b"".join([i.to_bytes(1, byteorder="little") for i in _bytes])
+                return None
+
+            return self.serialize(b_result)
+        else:
+            return self.serialize(self.current_max)[: self.trunc_length]

Review Comment:
   How could this logic be correct? It seems suspicious to me. I think that it's likely that the idea here was that this will never be called, but instead this should throw an exception or return None to signal the value cannot be truncated.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1285257847


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1039,335 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.values()).union({"INT96"})
+
+
+class StatsAggregator:
+    current_min: Any
+    current_max: Any
+    trunc_length: Optional[int]
+
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min = None
+        self.current_max = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        expected_physical_type = _PRIMITIVE_TO_PHYSICAL[iceberg_type]
+        if expected_physical_type != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {expected_physical_type}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        self.current_min = val if self.current_min is None else min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        self.current_max = val if self.current_max is None else max(val, self.current_max)
+
+    def get_min(self) -> bytes:

Review Comment:
   Style: unnecessary `get` in a method name.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1285257779


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1039,335 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.values()).union({"INT96"})
+
+
+class StatsAggregator:
+    current_min: Any
+    current_max: Any
+    trunc_length: Optional[int]
+
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min = None
+        self.current_max = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        expected_physical_type = _PRIMITIVE_TO_PHYSICAL[iceberg_type]
+        if expected_physical_type != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {expected_physical_type}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)

Review Comment:
   Is this needed because Arrow internally represents the UUID type as `fixed[16]`? If so, can't we just return the value directly since it's already bytes?



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] maxdebayser commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "maxdebayser (via GitHub)" <gi...@apache.org>.
maxdebayser commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1286025472


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1039,335 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.values()).union({"INT96"})
+
+
+class StatsAggregator:
+    current_min: Any
+    current_max: Any
+    trunc_length: Optional[int]
+
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min = None
+        self.current_max = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        expected_physical_type = _PRIMITIVE_TO_PHYSICAL[iceberg_type]
+        if expected_physical_type != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {expected_physical_type}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        self.current_min = val if self.current_min is None else min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        self.current_max = val if self.current_max is None else max(val, self.current_max)
+
+    def get_min(self) -> bytes:
+        return self.serialize(
+            self.current_min
+            if self.trunc_length is None
+            else TruncateTransform(width=self.trunc_length).transform(self.primitive_type)(self.current_min)
+        )
+
+    def get_max(self) -> Optional[bytes]:
+        if self.current_max is None:
+            return None
+
+        if self.primitive_type == StringType():
+            if type(self.current_max) != str:
+                raise ValueError("Expected the current_max to be a string")
+
+            s_result = self.current_max[: self.trunc_length]
+            if s_result != self.current_max:
+                chars = [*s_result]
+
+                for i in range(-1, -len(s_result) - 1, -1):
+                    try:
+                        to_inc = ord(chars[i])
+                        # will raise exception if the highest unicode code is reached
+                        _next = chr(to_inc + 1)
+                        chars[i] = _next
+                        return self.serialize("".join(chars))
+                    except ValueError:
+                        pass
+                return None  # didn't find a valid upper bound
+            return self.serialize(s_result)
+        elif self.primitive_type == BinaryType():
+            if type(self.current_max) != bytes:
+                raise ValueError("Expected the current_max to be bytes")
+            b_result = self.current_max[: self.trunc_length]
+            if b_result != self.current_max:
+                _bytes = [*b_result]
+                for i in range(-1, -len(b_result) - 1, -1):
+                    if _bytes[i] < 255:
+                        _bytes[i] += 1
+                        return b"".join([i.to_bytes(1, byteorder="little") for i in _bytes])
+                return None
+
+            return self.serialize(b_result)
+        else:
+            return self.serialize(self.current_max)[: self.trunc_length]
+
+
+DEFAULT_TRUNCATION_LENGTH = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+
+class MetricModeTypes(Enum):
+    TRUNCATE = "truncate"
+    NONE = "none"
+    COUNTS = "counts"
+    FULL = "full"
+
+
+DEFAULT_METRICS_MODE_KEY = "write.metadata.metrics.default"
+COLUMN_METRICS_MODE_KEY_PREFIX = "write.metadata.metrics.column"
+
+
+@dataclass(frozen=True)
+class MetricsMode(Singleton):
+    type: MetricModeTypes
+    length: Optional[int] = None
+
+
+def match_metrics_mode(mode: str) -> MetricsMode:
+    sanitized_mode = mode.lower()
+    if sanitized_mode.startswith("truncate"):
+        m = re.match(TRUNCATION_EXPR, mode, re.IGNORECASE)
+        if m:
+            length = int(m[1])
+            if length < 1:
+                raise ValueError("Truncation length must be larger than 0")
+            return MetricsMode(MetricModeTypes.TRUNCATE, int(m[1]))
+        else:
+            raise ValueError(f"Malformed truncate: {mode}")
+    elif sanitized_mode.startswith("none"):
+        return MetricsMode(MetricModeTypes.NONE)
+    elif sanitized_mode.startswith("counts"):
+        return MetricsMode(MetricModeTypes.COUNTS)
+    elif sanitized_mode.startswith("full"):
+        return MetricsMode(MetricModeTypes.FULL)
+    else:
+        raise ValueError(f"Unsupported metrics mode: {mode}")
+
+
+@dataclass(frozen=True)
+class StatisticsCollector:
+    field_id: int
+    iceberg_type: PrimitiveType
+    mode: MetricsMode
+    column_name: str
+
+
+class PyArrowStatisticsCollector(PreOrderSchemaVisitor[List[StatisticsCollector]]):
+    _field_id: int = 0
+    _schema: Schema
+    _properties: Dict[str, str]
+
+    def __init__(self, schema: Schema, properties: Dict[str, str]):
+        self._schema = schema
+        self._properties = properties
+
+    def schema(self, schema: Schema, struct_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        return struct_result()
+
+    def struct(
+        self, struct: StructType, field_results: List[Callable[[], List[StatisticsCollector]]]
+    ) -> List[StatisticsCollector]:
+        return list(chain(*[result() for result in field_results]))
+
+    def field(self, field: NestedField, field_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = field.field_id
+        result = field_result()
+        return result
+
+    def list(self, list_type: ListType, element_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = list_type.element_id
+        return element_result()
+
+    def map(
+        self,
+        map_type: MapType,
+        key_result: Callable[[], List[StatisticsCollector]],
+        value_result: Callable[[], List[StatisticsCollector]],
+    ) -> List[StatisticsCollector]:
+        self._field_id = map_type.key_id
+        k = key_result()
+        self._field_id = map_type.value_id
+        v = value_result()
+        return k + v
+
+    def primitive(self, primitive: PrimitiveType) -> List[StatisticsCollector]:
+        column_name = self._schema.find_column_name(self._field_id)
+        if column_name is None:
+            raise ValueError(f"Column for field {self._field_id} not found")
+
+        metrics_mode = MetricsMode(MetricModeTypes.TRUNCATE, DEFAULT_TRUNCATION_LENGTH)
+
+        default_mode = self._properties.get(DEFAULT_METRICS_MODE_KEY)
+        if default_mode:
+            metrics_mode = match_metrics_mode(default_mode)
+
+        col_mode = self._properties.get(f"{COLUMN_METRICS_MODE_KEY_PREFIX}.{column_name}")
+        if col_mode:
+            metrics_mode = match_metrics_mode(col_mode)
+
+        if (
+            not (isinstance(primitive, StringType) or isinstance(primitive, BinaryType))
+            and metrics_mode.type == MetricModeTypes.TRUNCATE
+        ):
+            metrics_mode = MetricsMode(MetricModeTypes.FULL)
+
+        return [StatisticsCollector(field_id=self._field_id, iceberg_type=primitive, mode=metrics_mode, column_name=column_name)]
+
+
+def fill_parquet_file_metadata(
+    df: DataFile,
+    parquet_metadata: pq.FileMetaData,
+    file_size: int,
+    table_metadata: TableMetadata,

Review Comment:
   Yes, the first iteration had this separation of concerns: https://github.com/apache/iceberg/blob/ce8a5df5d07b51dbee03c7d8d88dcff99ffbde02/python/pyiceberg/utils/file_stats.py#L121 . We can split it up again.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] maxdebayser commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "maxdebayser (via GitHub)" <gi...@apache.org>.
maxdebayser commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1286046035


##########
python/tests/io/test_pyarrow.py:
##########
@@ -1345,3 +1374,655 @@ def test_pyarrow_wrap_fsspec(example_task: FileScanTask, table_schema_simple: Sc
 bar: [[1,2,3]]
 baz: [[true,false,null]]"""
     )
+
+
+def construct_test_table() -> Tuple[Any, Any, Union[TableMetadataV1, TableMetadataV2]]:
+    table_metadata = {
+        "format-version": 2,
+        "location": "s3://bucket/test/location",
+        "last-column-id": 7,
+        "current-schema-id": 0,
+        "schemas": [
+            {
+                "type": "struct",
+                "schema-id": 0,
+                "fields": [
+                    {"id": 1, "name": "strings", "required": False, "type": "string"},
+                    {"id": 2, "name": "floats", "required": False, "type": "float"},
+                    {
+                        "id": 3,
+                        "name": "list",
+                        "required": False,
+                        "type": {"type": "list", "element-id": 5, "element": "long", "element-required": False},
+                    },
+                    {
+                        "id": 4,
+                        "name": "maps",
+                        "required": False,
+                        "type": {
+                            "type": "map",
+                            "key-id": 6,
+                            "key": "long",
+                            "value-id": 7,
+                            "value": "long",
+                            "value-required": False,
+                        },
+                    },
+                ],
+            },
+        ],
+        "default-spec-id": 0,
+        "partition-specs": [{"spec-id": 0, "fields": []}],
+        "properties": {},
+    }
+
+    table_metadata = TableMetadataUtil.parse_obj(table_metadata)
+    arrow_schema = schema_to_pyarrow(table_metadata.schemas[0])
+
+    _strings = ["zzzzzzzzzzzzzzzzzzzz", "rrrrrrrrrrrrrrrrrrrr", None, "aaaaaaaaaaaaaaaaaaaa"]
+
+    _floats = [3.14, math.nan, 1.69, 100]
+
+    _list = [[1, 2, 3], [4, 5, 6], None, [7, 8, 9]]
+
+    _maps: List[Optional[Dict[int, int]]] = [
+        {1: 2, 3: 4},
+        None,
+        {5: 6},
+        {},
+    ]
+
+    table = pa.Table.from_pydict(
+        {
+            "strings": _strings,
+            "floats": _floats,
+            "list": _list,
+            "maps": _maps,
+        },
+        schema=arrow_schema,
+    )
+    metadata_collector: List[Any] = []
+
+    with pa.BufferOutputStream() as f:
+        with pq.ParquetWriter(f, table.schema, metadata_collector=metadata_collector) as writer:
+            writer.write_table(table)
+
+        return f.getvalue(), metadata_collector[0], table_metadata
+
+
+def test_record_count() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.record_count == 4
+
+
+def test_file_size() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.file_size_in_bytes == len(file_bytes)
+
+
+def test_value_counts() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.value_counts) == 5
+    assert datafile.value_counts[1] == 4
+    assert datafile.value_counts[2] == 4
+    assert datafile.value_counts[5] == 10  # 3 lists with 3 items and a None value
+    assert datafile.value_counts[6] == 5
+    assert datafile.value_counts[7] == 5
+
+
+def test_column_sizes() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.column_sizes) == 5
+    # these values are an artifact of how the write_table encodes the columns

Review Comment:
   Yes, that's better.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1288323847


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1039,335 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.values()).union({"INT96"})
+
+
+class StatsAggregator:
+    current_min: Any
+    current_max: Any
+    trunc_length: Optional[int]
+
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min = None
+        self.current_max = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        expected_physical_type = _PRIMITIVE_TO_PHYSICAL[iceberg_type]
+        if expected_physical_type != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {expected_physical_type}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)

Review Comment:
   Yes, this will be done in https://github.com/apache/iceberg/pull/8267 including testing



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] maxdebayser commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "maxdebayser (via GitHub)" <gi...@apache.org>.
maxdebayser commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1289386763


##########
python/tests/io/test_pyarrow.py:
##########
@@ -1345,3 +1374,655 @@ def test_pyarrow_wrap_fsspec(example_task: FileScanTask, table_schema_simple: Sc
 bar: [[1,2,3]]
 baz: [[true,false,null]]"""
     )
+
+
+def construct_test_table() -> Tuple[Any, Any, Union[TableMetadataV1, TableMetadataV2]]:
+    table_metadata = {
+        "format-version": 2,
+        "location": "s3://bucket/test/location",
+        "last-column-id": 7,
+        "current-schema-id": 0,
+        "schemas": [
+            {
+                "type": "struct",
+                "schema-id": 0,
+                "fields": [
+                    {"id": 1, "name": "strings", "required": False, "type": "string"},
+                    {"id": 2, "name": "floats", "required": False, "type": "float"},
+                    {
+                        "id": 3,
+                        "name": "list",
+                        "required": False,
+                        "type": {"type": "list", "element-id": 5, "element": "long", "element-required": False},
+                    },
+                    {
+                        "id": 4,
+                        "name": "maps",
+                        "required": False,
+                        "type": {
+                            "type": "map",
+                            "key-id": 6,
+                            "key": "long",
+                            "value-id": 7,
+                            "value": "long",
+                            "value-required": False,
+                        },
+                    },
+                ],
+            },
+        ],
+        "default-spec-id": 0,
+        "partition-specs": [{"spec-id": 0, "fields": []}],
+        "properties": {},
+    }
+
+    table_metadata = TableMetadataUtil.parse_obj(table_metadata)
+    arrow_schema = schema_to_pyarrow(table_metadata.schemas[0])
+
+    _strings = ["zzzzzzzzzzzzzzzzzzzz", "rrrrrrrrrrrrrrrrrrrr", None, "aaaaaaaaaaaaaaaaaaaa"]
+
+    _floats = [3.14, math.nan, 1.69, 100]
+
+    _list = [[1, 2, 3], [4, 5, 6], None, [7, 8, 9]]
+
+    _maps: List[Optional[Dict[int, int]]] = [
+        {1: 2, 3: 4},
+        None,
+        {5: 6},
+        {},
+    ]
+
+    table = pa.Table.from_pydict(
+        {
+            "strings": _strings,
+            "floats": _floats,
+            "list": _list,
+            "maps": _maps,
+        },
+        schema=arrow_schema,
+    )
+    metadata_collector: List[Any] = []
+
+    with pa.BufferOutputStream() as f:
+        with pq.ParquetWriter(f, table.schema, metadata_collector=metadata_collector) as writer:
+            writer.write_table(table)
+
+        return f.getvalue(), metadata_collector[0], table_metadata
+
+
+def test_record_count() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.record_count == 4
+
+
+def test_file_size() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.file_size_in_bytes == len(file_bytes)
+
+
+def test_value_counts() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.value_counts) == 5
+    assert datafile.value_counts[1] == 4
+    assert datafile.value_counts[2] == 4
+    assert datafile.value_counts[5] == 10  # 3 lists with 3 items and a None value
+    assert datafile.value_counts[6] == 5

Review Comment:
   Seems to be consistent with what we're seeing in python



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1289075132


##########
python/tests/io/test_pyarrow.py:
##########
@@ -1345,3 +1374,655 @@ def test_pyarrow_wrap_fsspec(example_task: FileScanTask, table_schema_simple: Sc
 bar: [[1,2,3]]
 baz: [[true,false,null]]"""
     )
+
+
+def construct_test_table() -> Tuple[Any, Any, Union[TableMetadataV1, TableMetadataV2]]:
+    table_metadata = {
+        "format-version": 2,
+        "location": "s3://bucket/test/location",
+        "last-column-id": 7,
+        "current-schema-id": 0,
+        "schemas": [
+            {
+                "type": "struct",
+                "schema-id": 0,
+                "fields": [
+                    {"id": 1, "name": "strings", "required": False, "type": "string"},
+                    {"id": 2, "name": "floats", "required": False, "type": "float"},
+                    {
+                        "id": 3,
+                        "name": "list",
+                        "required": False,
+                        "type": {"type": "list", "element-id": 5, "element": "long", "element-required": False},
+                    },
+                    {
+                        "id": 4,
+                        "name": "maps",
+                        "required": False,
+                        "type": {
+                            "type": "map",
+                            "key-id": 6,
+                            "key": "long",
+                            "value-id": 7,
+                            "value": "long",
+                            "value-required": False,
+                        },
+                    },
+                ],
+            },
+        ],
+        "default-spec-id": 0,
+        "partition-specs": [{"spec-id": 0, "fields": []}],
+        "properties": {},
+    }
+
+    table_metadata = TableMetadataUtil.parse_obj(table_metadata)
+    arrow_schema = schema_to_pyarrow(table_metadata.schemas[0])
+
+    _strings = ["zzzzzzzzzzzzzzzzzzzz", "rrrrrrrrrrrrrrrrrrrr", None, "aaaaaaaaaaaaaaaaaaaa"]
+
+    _floats = [3.14, math.nan, 1.69, 100]
+
+    _list = [[1, 2, 3], [4, 5, 6], None, [7, 8, 9]]
+
+    _maps: List[Optional[Dict[int, int]]] = [
+        {1: 2, 3: 4},
+        None,
+        {5: 6},
+        {},
+    ]
+
+    table = pa.Table.from_pydict(
+        {
+            "strings": _strings,
+            "floats": _floats,
+            "list": _list,
+            "maps": _maps,
+        },
+        schema=arrow_schema,
+    )
+    metadata_collector: List[Any] = []
+
+    with pa.BufferOutputStream() as f:
+        with pq.ParquetWriter(f, table.schema, metadata_collector=metadata_collector) as writer:
+            writer.write_table(table)
+
+        return f.getvalue(), metadata_collector[0], table_metadata
+
+
+def test_record_count() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.record_count == 4
+
+
+def test_file_size() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.file_size_in_bytes == len(file_bytes)
+
+
+def test_value_counts() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.value_counts) == 5
+    assert datafile.value_counts[1] == 4
+    assert datafile.value_counts[2] == 4
+    assert datafile.value_counts[5] == 10  # 3 lists with 3 items and a None value
+    assert datafile.value_counts[6] == 5

Review Comment:
   ```sql
   CREATE TABLE default.arrays
   SELECT
       array(1, 2, 3, null) as with_a_null,
       array(1, 2, cast('NaN' as double), null) as with_null_and_nan,
       array(1, 2, 3, cast('NaN' as double)) as with_a_nan
   ```
   
   Schema:
   ```json
   {
   	"type": "struct",
   	"schema-id": 0,
   	"fields": [{
   		"id": 1,
   		"name": "with_a_null",
   		"required": false,
   		"type": {
   			"type": "list",
   			"element-id": 4,
   			"element": "int",
   			"element-required": false
   		}
   	}, {
   		"id": 2,
   		"name": "with_null_and_nan",
   		"required": false,
   		"type": {
   			"type": "list",
   			"element-id": 5,
   			"element": "double",
   			"element-required": false
   		}
   	}, {
   		"id": 3,
   		"name": "with_a_nan",
   		"required": false,
   		"type": {
   			"type": "list",
   			"element-id": 6,
   			"element": "double",
   			"element-required": false
   		}
   	}]
   }
   ```
   
   The missing upper- and lower bound seems to be an issue with Spark. It does generates the doubles:
   
   ```json
   {
   	"status": 1,
   	"snapshot_id": {
   		"long": 3911973389432551915
   	},
   	"data_file": {
   		"file_path": "s3://warehouse/default/arrays/data/00000-0-1e0d4a20-4473-450e-9b1e-9196b5edac66-00001.parquet",
   		"file_format": "PARQUET",
   		"partition": {},
   		"record_count": 1,
   		"file_size_in_bytes": 1310,
   		"block_size_in_bytes": 67108864,
   		"column_sizes": {
   			"array": [{
   				"key": 4,
   				"value": 60
   			}, {
   				"key": 5,
   				"value": 63
   			}, {
   				"key": 6,
   				"value": 66
   			}]
   		},
   		"value_counts": {
   			"array": [{
   				"key": 4,
   				"value": 4
   			}, {
   				"key": 5,
   				"value": 4
   			}, {
   				"key": 6,
   				"value": 4
   			}]
   		},
   		"null_value_counts": {
   			"array": [{
   				"key": 4,
   				"value": 1
   			}, {
   				"key": 5,
   				"value": 1
   			}, {
   				"key": 6,
   				"value": 0
   			}]
   		},
   		"nan_value_counts": {
   			"array": [{
   				"key": 5,
   				"value": 1
   			}, {
   				"key": 6,
   				"value": 1
   			}]
   		},
   		"lower_bounds": {
   			"array": [{
   				"key": 5,
   				"value": "\u0000\u0000\u0000\u0000\u0000\u0000ð?"
   			}, {
   				"key": 6,
   				"value": "\u0000\u0000\u0000\u0000\u0000\u0000ð?"
   			}]
   		},
   		"upper_bounds": {
   			"array": [{
   				"key": 5,
   				"value": "\u0000\u0000\u0000\u0000\u0000\u0000\u0000@"
   			}, {
   				"key": 6,
   				"value": "\u0000\u0000\u0000\u0000\u0000\u0000\b@"
   			}]
   		},
   		"key_metadata": null,
   		"split_offsets": {
   			"array": [4]
   		},
   		"sort_order_id": {
   			"int": 0
   		}
   	}
   }
   ```



##########
python/tests/io/test_pyarrow.py:
##########
@@ -1345,3 +1374,655 @@ def test_pyarrow_wrap_fsspec(example_task: FileScanTask, table_schema_simple: Sc
 bar: [[1,2,3]]
 baz: [[true,false,null]]"""
     )
+
+
+def construct_test_table() -> Tuple[Any, Any, Union[TableMetadataV1, TableMetadataV2]]:
+    table_metadata = {
+        "format-version": 2,
+        "location": "s3://bucket/test/location",
+        "last-column-id": 7,
+        "current-schema-id": 0,
+        "schemas": [
+            {
+                "type": "struct",
+                "schema-id": 0,
+                "fields": [
+                    {"id": 1, "name": "strings", "required": False, "type": "string"},
+                    {"id": 2, "name": "floats", "required": False, "type": "float"},
+                    {
+                        "id": 3,
+                        "name": "list",
+                        "required": False,
+                        "type": {"type": "list", "element-id": 5, "element": "long", "element-required": False},
+                    },
+                    {
+                        "id": 4,
+                        "name": "maps",
+                        "required": False,
+                        "type": {
+                            "type": "map",
+                            "key-id": 6,
+                            "key": "long",
+                            "value-id": 7,
+                            "value": "long",
+                            "value-required": False,
+                        },
+                    },
+                ],
+            },
+        ],
+        "default-spec-id": 0,
+        "partition-specs": [{"spec-id": 0, "fields": []}],
+        "properties": {},
+    }
+
+    table_metadata = TableMetadataUtil.parse_obj(table_metadata)
+    arrow_schema = schema_to_pyarrow(table_metadata.schemas[0])
+
+    _strings = ["zzzzzzzzzzzzzzzzzzzz", "rrrrrrrrrrrrrrrrrrrr", None, "aaaaaaaaaaaaaaaaaaaa"]
+
+    _floats = [3.14, math.nan, 1.69, 100]
+
+    _list = [[1, 2, 3], [4, 5, 6], None, [7, 8, 9]]
+
+    _maps: List[Optional[Dict[int, int]]] = [
+        {1: 2, 3: 4},
+        None,
+        {5: 6},
+        {},
+    ]
+
+    table = pa.Table.from_pydict(
+        {
+            "strings": _strings,
+            "floats": _floats,
+            "list": _list,
+            "maps": _maps,
+        },
+        schema=arrow_schema,
+    )
+    metadata_collector: List[Any] = []
+
+    with pa.BufferOutputStream() as f:
+        with pq.ParquetWriter(f, table.schema, metadata_collector=metadata_collector) as writer:
+            writer.write_table(table)
+
+        return f.getvalue(), metadata_collector[0], table_metadata
+
+
+def test_record_count() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.record_count == 4
+
+
+def test_file_size() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.file_size_in_bytes == len(file_bytes)
+
+
+def test_value_counts() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.value_counts) == 5
+    assert datafile.value_counts[1] == 4
+    assert datafile.value_counts[2] == 4
+    assert datafile.value_counts[5] == 10  # 3 lists with 3 items and a None value
+    assert datafile.value_counts[6] == 5

Review Comment:
   ```sql
   CREATE TABLE default.arrays
   SELECT
       array(1, 2, 3, null) as with_a_null,
       array(1, 2, cast('NaN' as double), null) as with_null_and_nan,
       array(1, 2, 3, cast('NaN' as double)) as with_a_nan
   ```
   
   Schema:
   ```json
   {
   	"type": "struct",
   	"schema-id": 0,
   	"fields": [{
   		"id": 1,
   		"name": "with_a_null",
   		"required": false,
   		"type": {
   			"type": "list",
   			"element-id": 4,
   			"element": "int",
   			"element-required": false
   		}
   	}, {
   		"id": 2,
   		"name": "with_null_and_nan",
   		"required": false,
   		"type": {
   			"type": "list",
   			"element-id": 5,
   			"element": "double",
   			"element-required": false
   		}
   	}, {
   		"id": 3,
   		"name": "with_a_nan",
   		"required": false,
   		"type": {
   			"type": "list",
   			"element-id": 6,
   			"element": "double",
   			"element-required": false
   		}
   	}]
   }
   ```
   
   The missing upper- and lower bound seems to be an issue with Spark. It does generates the doubles:
   
   ```json
   {
   	"status": 1,
   	"snapshot_id": {
   		"long": 3911973389432551915
   	},
   	"data_file": {
   		"file_path": "s3://warehouse/default/arrays/data/00000-0-1e0d4a20-4473-450e-9b1e-9196b5edac66-00001.parquet",
   		"file_format": "PARQUET",
   		"partition": {},
   		"record_count": 1,
   		"file_size_in_bytes": 1310,
   		"block_size_in_bytes": 67108864,
   		"column_sizes": {
   			"array": [{
   				"key": 4,
   				"value": 60
   			}, {
   				"key": 5,
   				"value": 63
   			}, {
   				"key": 6,
   				"value": 66
   			}]
   		},
   		"value_counts": {
   			"array": [{
   				"key": 4,
   				"value": 4
   			}, {
   				"key": 5,
   				"value": 4
   			}, {
   				"key": 6,
   				"value": 4
   			}]
   		},
   		"null_value_counts": {
   			"array": [{
   				"key": 4,
   				"value": 1
   			}, {
   				"key": 5,
   				"value": 1
   			}, {
   				"key": 6,
   				"value": 0
   			}]
   		},
   		"nan_value_counts": {
   			"array": [{
   				"key": 5,
   				"value": 1
   			}, {
   				"key": 6,
   				"value": 1
   			}]
   		},
   		"lower_bounds": {
   			"array": [{
   				"key": 5,
   				"value": "\u0000\u0000\u0000\u0000\u0000\u0000ð?"
   			}, {
   				"key": 6,
   				"value": "\u0000\u0000\u0000\u0000\u0000\u0000ð?"
   			}]
   		},
   		"upper_bounds": {
   			"array": [{
   				"key": 5,
   				"value": "\u0000\u0000\u0000\u0000\u0000\u0000\u0000@"
   			}, {
   				"key": 6,
   				"value": "\u0000\u0000\u0000\u0000\u0000\u0000\b@"
   			}]
   		},
   		"key_metadata": null,
   		"split_offsets": {
   			"array": [4]
   		},
   		"sort_order_id": {
   			"int": 0
   		}
   	}
   }
   ```



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on pull request #7831: Python: Compute parquet stats

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#issuecomment-1623188606

   @maxdebayser can you rebase against lastest master?


-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] maxdebayser commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "maxdebayser (via GitHub)" <gi...@apache.org>.
maxdebayser commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1258878681


##########
python/tests/utils/test_file_stats.py:
##########
@@ -0,0 +1,361 @@
+# 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.
+
+
+import math
+import struct
+from tempfile import TemporaryDirectory
+from typing import Any, List
+
+import pyarrow as pa
+import pyarrow.parquet as pq
+
+from pyiceberg.manifest import DataFile
+from pyiceberg.schema import Schema
+from pyiceberg.utils.file_stats import BOUND_TRUNCATED_LENGHT, fill_parquet_file_metadata, parquet_schema_to_ids
+
+
+def construct_test_table() -> pa.Buffer:
+    schema = pa.schema(
+        [pa.field("strings", pa.string()), pa.field("floats", pa.float64()), pa.field("list", pa.list_(pa.int64()))]

Review Comment:
   Sure



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] maxdebayser commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "maxdebayser (via GitHub)" <gi...@apache.org>.
maxdebayser commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1262884747


##########
python/pyiceberg/avro/__init__.py:
##########
@@ -16,5 +16,8 @@
 # under the License.
 import struct
 
+STRUCT_BOOL = struct.Struct("?")

Review Comment:
   I was reverting this change when I realized I'd started using it in the unit tests, so I thought it would be better to keep the Code DRY.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1262637285


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1013,3 +1025,271 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+class StatsAggregator:
+    def __init__(self, type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.trunc_length = trunc_length
+        self.primitive_type: Optional[PrimitiveType] = None
+
+        if type_string == "BOOLEAN":
+            self.primitive_type = BooleanType()
+        elif type_string == "INT32":
+            self.primitive_type = IntegerType()
+        elif type_string == "INT64":
+            self.primitive_type = LongType()
+        elif type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+        elif type_string == "FLOAT":
+            self.primitive_type = FloatType()
+        elif type_string == "DOUBLE":
+            self.primitive_type = DoubleType()
+        elif type_string == "BYTE_ARRAY":
+            self.primitive_type = BinaryType()
+        elif type_string == "FIXED_LEN_BYTE_ARRAY":
+            self.primitive_type = BinaryType()
+        else:
+            raise AssertionError(f"Unknown physical type {type_string}")
+
+    def serialize(self, value: Any) -> bytes:
+        if type(value) == str:
+            value = value.encode()
+        assert self.primitive_type is not None  # appease mypy
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        if self.current_min is None:
+            self.current_min = val
+        else:
+            self.current_min = min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        if self.current_max is None:
+            self.current_max = val
+        else:
+            self.current_max = max(self.current_max, val)
+
+    def get_min(self) -> bytes:
+        return self.serialize(self.current_min)[: self.trunc_length]
+
+    def get_max(self) -> bytes:
+        return self.serialize(self.current_max)[: self.trunc_length]
+
+
+DEFAULT_TRUNCATION_LENGHT = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+
+class MetricModeTypes(Enum):
+    TRUNCATE = "truncate"
+    NONE = "none"
+    COUNTS = "counts"
+    FULL = "full"
+
+
+DEFAULT_METRICS_MODE_KEY = "write.metadata.metrics.default"
+COLUMN_METRICS_MODE_KEY = "write.metadata.metrics.column"
+
+
+@dataclass(frozen=True)
+class MetricsMode(Singleton):
+    type: MetricModeTypes
+    length: Optional[int] = None
+
+
+def match_metrics_mode(mode: str) -> MetricsMode:
+    m = re.match(TRUNCATION_EXPR, mode, re.IGNORECASE)
+    if m:
+        length = int(m[1])
+        if length < 1:
+            raise AssertionError("Truncation length must be larger than 0")
+        return MetricsMode(MetricModeTypes.TRUNCATE, int(m[1]))
+    elif re.match("^none$", mode, re.IGNORECASE):
+        return MetricsMode(MetricModeTypes.NONE)
+    elif re.match("^counts$", mode, re.IGNORECASE):
+        return MetricsMode(MetricModeTypes.COUNTS)
+    elif re.match("^full$", mode, re.IGNORECASE):
+        return MetricsMode(MetricModeTypes.FULL)
+    else:
+        raise AssertionError(f"Unsupported metrics mode {mode}")
+
+
+@dataclass(frozen=True)
+class StatisticsCollector:
+    field_id: int
+    iceberg_type: PrimitiveType
+    mode: MetricsMode
+    column_name: str
+
+
+class PyArrowStatisticsCollector(PreOrderSchemaVisitor[List[StatisticsCollector]]):
+    _field_id = 0
+    _schema: Schema
+    _properties: Dict[str, str]
+
+    def __init__(self, schema: Schema, properties: Dict[str, str]):
+        self._schema = schema
+        self._properties = properties
+
+    def schema(self, schema: Schema, struct_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        return struct_result()
+
+    def struct(
+        self, struct: StructType, field_results: List[Callable[[], List[StatisticsCollector]]]
+    ) -> List[StatisticsCollector]:
+        return list(chain(*[result() for result in field_results]))
+
+    def field(self, field: NestedField, field_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = field.field_id
+        result = field_result()
+        return result
+
+    def list(self, list_type: ListType, element_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = list_type.element_id
+        return element_result()
+
+    def map(
+        self,
+        map_type: MapType,
+        key_result: Callable[[], List[StatisticsCollector]],
+        value_result: Callable[[], List[StatisticsCollector]],
+    ) -> List[StatisticsCollector]:
+        self._field_id = map_type.key_id
+        k = key_result()
+        self._field_id = map_type.value_id
+        v = value_result()
+        return k + v
+
+    def primitive(self, primitive: PrimitiveType) -> List[StatisticsCollector]:
+        column_name = self._schema.find_column_name(self._field_id)
+        assert column_name is not None, f"Column for field {self._field_id} not found"
+
+        metrics_mode = MetricsMode(MetricModeTypes.TRUNCATE, DEFAULT_TRUNCATION_LENGHT)
+
+        default_mode = self._properties.get(DEFAULT_METRICS_MODE_KEY)
+        if default_mode:
+            metrics_mode = match_metrics_mode(default_mode)
+
+        col_mode = self._properties.get(f"{COLUMN_METRICS_MODE_KEY}.{column_name}")
+        if col_mode:
+            metrics_mode = match_metrics_mode(col_mode)
+
+        return [StatisticsCollector(field_id=self._field_id, iceberg_type=primitive, mode=metrics_mode, column_name=column_name)]
+
+
+def fill_parquet_file_metadata(
+    df: DataFile,
+    parquet_metadata: pq.FileMetaData,
+    file_size: int,
+    table_metadata: TableMetadata,
+) -> None:
+    """
+    Computes and fills the following fields of the DataFile object.
+
+    - file_format
+    - record_count
+    - file_size_in_bytes
+    - column_sizes
+    - value_counts
+    - null_value_counts
+    - nan_value_counts
+    - lower_bounds
+    - upper_bounds
+    - split_offsets
+
+    Args:
+        df (DataFile): A DataFile object representing the Parquet file for which metadata is to be filled.
+        parquet_metadata (pyarrow.parquet.FileMetaData): A pyarrow metadata object.
+        file_size (int): The total compressed file size cannot be retrieved from the metadata and hence has to
+            be passed here. Depending on the kind of file system and pyarrow library call used, different
+            ways to obtain this value might be appropriate.
+        table_metadata (pyiceberg.table.metadata.TableMetadata): The Iceberg table metadata. It is required to
+            compute the mapping if column position to iceberg schema type id. It's also used to set the mode
+            for column metrics collection
+    """
+    schema = next(filter(lambda s: s.schema_id == table_metadata.current_schema_id, table_metadata.schemas))
+
+    stats_columns = pre_order_visit(schema, PyArrowStatisticsCollector(schema, table_metadata.properties))
+    assert parquet_metadata.num_columns == len(
+        stats_columns
+    ), f"Number of columns in metadata ({len(stats_columns)}) is different from the number of columns in pyarrow table ({parquet_metadata.num_columns})"
+
+    col_index_2_id = {i: stat.field_id for i, stat in enumerate(stats_columns)}
+
+    column_sizes: Dict[int, int] = {}
+    value_counts: Dict[int, int] = {}
+    split_offsets: List[int] = []
+
+    null_value_counts: Dict[int, int] = {}
+    nan_value_counts: Dict[int, int] = {}
+
+    col_aggs = {}
+
+    for r in range(parquet_metadata.num_row_groups):
+        # References:
+        # https://github.com/apache/iceberg/blob/fc381a81a1fdb8f51a0637ca27cd30673bd7aad3/parquet/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java#L232
+        # https://github.com/apache/parquet-mr/blob/ac29db4611f86a07cc6877b416aa4b183e09b353/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/ColumnChunkMetaData.java#L184
+
+        row_group = parquet_metadata.row_group(r)
+
+        data_offset = row_group.column(0).data_page_offset
+        dictionary_offset = row_group.column(0).dictionary_page_offset
+
+        if row_group.column(0).has_dictionary_page and dictionary_offset < data_offset:
+            split_offsets.append(dictionary_offset)
+        else:
+            split_offsets.append(data_offset)
+
+        for c in range(parquet_metadata.num_columns):

Review Comment:
   I think we can get rid of the `col_index_2_id` dict altogether:
   ```suggestion
           for idx, stat_col in enumerate(stats_columns):
   ```
   This way we have all the information that we need. 



##########
python/pyiceberg/avro/__init__.py:
##########
@@ -16,5 +16,8 @@
 # under the License.
 import struct
 
+STRUCT_BOOL = struct.Struct("?")

Review Comment:
   We can revert the changes in this file, now we use `to_bytes`



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1013,3 +1025,271 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+class StatsAggregator:
+    def __init__(self, type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.trunc_length = trunc_length
+        self.primitive_type: Optional[PrimitiveType] = None
+
+        if type_string == "BOOLEAN":
+            self.primitive_type = BooleanType()
+        elif type_string == "INT32":
+            self.primitive_type = IntegerType()
+        elif type_string == "INT64":
+            self.primitive_type = LongType()
+        elif type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+        elif type_string == "FLOAT":
+            self.primitive_type = FloatType()
+        elif type_string == "DOUBLE":
+            self.primitive_type = DoubleType()
+        elif type_string == "BYTE_ARRAY":
+            self.primitive_type = BinaryType()
+        elif type_string == "FIXED_LEN_BYTE_ARRAY":
+            self.primitive_type = BinaryType()
+        else:
+            raise AssertionError(f"Unknown physical type {type_string}")
+
+    def serialize(self, value: Any) -> bytes:
+        if type(value) == str:
+            value = value.encode()
+        assert self.primitive_type is not None  # appease mypy
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        if self.current_min is None:
+            self.current_min = val
+        else:
+            self.current_min = min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        if self.current_max is None:
+            self.current_max = val
+        else:
+            self.current_max = max(self.current_max, val)
+
+    def get_min(self) -> bytes:
+        return self.serialize(self.current_min)[: self.trunc_length]
+
+    def get_max(self) -> bytes:
+        return self.serialize(self.current_max)[: self.trunc_length]
+
+
+DEFAULT_TRUNCATION_LENGHT = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+
+class MetricModeTypes(Enum):
+    TRUNCATE = "truncate"
+    NONE = "none"
+    COUNTS = "counts"
+    FULL = "full"
+
+
+DEFAULT_METRICS_MODE_KEY = "write.metadata.metrics.default"
+COLUMN_METRICS_MODE_KEY = "write.metadata.metrics.column"
+
+
+@dataclass(frozen=True)
+class MetricsMode(Singleton):
+    type: MetricModeTypes
+    length: Optional[int] = None
+
+
+def match_metrics_mode(mode: str) -> MetricsMode:
+    m = re.match(TRUNCATION_EXPR, mode, re.IGNORECASE)
+    if m:
+        length = int(m[1])
+        if length < 1:
+            raise AssertionError("Truncation length must be larger than 0")
+        return MetricsMode(MetricModeTypes.TRUNCATE, int(m[1]))
+    elif re.match("^none$", mode, re.IGNORECASE):
+        return MetricsMode(MetricModeTypes.NONE)
+    elif re.match("^counts$", mode, re.IGNORECASE):
+        return MetricsMode(MetricModeTypes.COUNTS)
+    elif re.match("^full$", mode, re.IGNORECASE):
+        return MetricsMode(MetricModeTypes.FULL)
+    else:
+        raise AssertionError(f"Unsupported metrics mode {mode}")
+
+
+@dataclass(frozen=True)
+class StatisticsCollector:
+    field_id: int
+    iceberg_type: PrimitiveType
+    mode: MetricsMode
+    column_name: str
+
+
+class PyArrowStatisticsCollector(PreOrderSchemaVisitor[List[StatisticsCollector]]):
+    _field_id = 0
+    _schema: Schema
+    _properties: Dict[str, str]
+
+    def __init__(self, schema: Schema, properties: Dict[str, str]):
+        self._schema = schema
+        self._properties = properties
+
+    def schema(self, schema: Schema, struct_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        return struct_result()
+
+    def struct(
+        self, struct: StructType, field_results: List[Callable[[], List[StatisticsCollector]]]
+    ) -> List[StatisticsCollector]:
+        return list(chain(*[result() for result in field_results]))
+
+    def field(self, field: NestedField, field_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = field.field_id
+        result = field_result()
+        return result
+
+    def list(self, list_type: ListType, element_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = list_type.element_id
+        return element_result()
+
+    def map(
+        self,
+        map_type: MapType,
+        key_result: Callable[[], List[StatisticsCollector]],
+        value_result: Callable[[], List[StatisticsCollector]],
+    ) -> List[StatisticsCollector]:
+        self._field_id = map_type.key_id
+        k = key_result()
+        self._field_id = map_type.value_id
+        v = value_result()
+        return k + v
+
+    def primitive(self, primitive: PrimitiveType) -> List[StatisticsCollector]:
+        column_name = self._schema.find_column_name(self._field_id)
+        assert column_name is not None, f"Column for field {self._field_id} not found"
+
+        metrics_mode = MetricsMode(MetricModeTypes.TRUNCATE, DEFAULT_TRUNCATION_LENGHT)
+
+        default_mode = self._properties.get(DEFAULT_METRICS_MODE_KEY)
+        if default_mode:
+            metrics_mode = match_metrics_mode(default_mode)
+
+        col_mode = self._properties.get(f"{COLUMN_METRICS_MODE_KEY}.{column_name}")
+        if col_mode:
+            metrics_mode = match_metrics_mode(col_mode)
+
+        return [StatisticsCollector(field_id=self._field_id, iceberg_type=primitive, mode=metrics_mode, column_name=column_name)]
+
+
+def fill_parquet_file_metadata(
+    df: DataFile,
+    parquet_metadata: pq.FileMetaData,
+    file_size: int,
+    table_metadata: TableMetadata,
+) -> None:
+    """
+    Computes and fills the following fields of the DataFile object.
+
+    - file_format
+    - record_count
+    - file_size_in_bytes
+    - column_sizes
+    - value_counts
+    - null_value_counts
+    - nan_value_counts
+    - lower_bounds
+    - upper_bounds
+    - split_offsets
+
+    Args:
+        df (DataFile): A DataFile object representing the Parquet file for which metadata is to be filled.
+        parquet_metadata (pyarrow.parquet.FileMetaData): A pyarrow metadata object.
+        file_size (int): The total compressed file size cannot be retrieved from the metadata and hence has to
+            be passed here. Depending on the kind of file system and pyarrow library call used, different
+            ways to obtain this value might be appropriate.
+        table_metadata (pyiceberg.table.metadata.TableMetadata): The Iceberg table metadata. It is required to
+            compute the mapping if column position to iceberg schema type id. It's also used to set the mode
+            for column metrics collection
+    """
+    schema = next(filter(lambda s: s.schema_id == table_metadata.current_schema_id, table_metadata.schemas))
+
+    stats_columns = pre_order_visit(schema, PyArrowStatisticsCollector(schema, table_metadata.properties))
+    assert parquet_metadata.num_columns == len(
+        stats_columns
+    ), f"Number of columns in metadata ({len(stats_columns)}) is different from the number of columns in pyarrow table ({parquet_metadata.num_columns})"
+
+    col_index_2_id = {i: stat.field_id for i, stat in enumerate(stats_columns)}
+
+    column_sizes: Dict[int, int] = {}
+    value_counts: Dict[int, int] = {}
+    split_offsets: List[int] = []
+
+    null_value_counts: Dict[int, int] = {}
+    nan_value_counts: Dict[int, int] = {}
+
+    col_aggs = {}
+
+    for r in range(parquet_metadata.num_row_groups):
+        # References:
+        # https://github.com/apache/iceberg/blob/fc381a81a1fdb8f51a0637ca27cd30673bd7aad3/parquet/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java#L232
+        # https://github.com/apache/parquet-mr/blob/ac29db4611f86a07cc6877b416aa4b183e09b353/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/ColumnChunkMetaData.java#L184
+
+        row_group = parquet_metadata.row_group(r)
+
+        data_offset = row_group.column(0).data_page_offset
+        dictionary_offset = row_group.column(0).dictionary_page_offset
+
+        if row_group.column(0).has_dictionary_page and dictionary_offset < data_offset:
+            split_offsets.append(dictionary_offset)
+        else:
+            split_offsets.append(data_offset)
+
+        for c in range(parquet_metadata.num_columns):
+            col_id = col_index_2_id[c]
+
+            column = row_group.column(c)
+
+            column_sizes[col_id] = column_sizes.get(col_id, 0) + column.total_compressed_size
+
+            metrics_mode = stats_columns[c].mode
+
+            if metrics_mode == MetricsMode(MetricModeTypes.NONE):
+                continue
+
+            value_counts[col_id] = value_counts.get(col_id, 0) + column.num_values
+
+            if column.is_stats_set:

Review Comment:
   When this isn't set, we probably want to emit a `logger.warn("PyArrow statistics missing when writing file")`



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1264728263


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1013,3 +1025,271 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+class StatsAggregator:
+    def __init__(self, type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.trunc_length = trunc_length
+        self.primitive_type: Optional[PrimitiveType] = None
+
+        if type_string == "BOOLEAN":
+            self.primitive_type = BooleanType()
+        elif type_string == "INT32":
+            self.primitive_type = IntegerType()
+        elif type_string == "INT64":
+            self.primitive_type = LongType()
+        elif type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+        elif type_string == "FLOAT":
+            self.primitive_type = FloatType()
+        elif type_string == "DOUBLE":
+            self.primitive_type = DoubleType()
+        elif type_string == "BYTE_ARRAY":
+            self.primitive_type = BinaryType()
+        elif type_string == "FIXED_LEN_BYTE_ARRAY":
+            self.primitive_type = BinaryType()
+        else:
+            raise AssertionError(f"Unknown physical type {type_string}")

Review Comment:
   ```suggestion
   class StatsAggregator:
       current_min: Any
       current_max: Any
       trunc_length: Optional[int]
       primitive_type: PrimitiveType
   
       def __init__(self, iceberg_type: PrimitiveType, trunc_length: Optional[int] = None) -> None:
           self.current_min: Any = None
           self.current_max: Any = None
           self.trunc_length = trunc_length
           self.primitive_type: PrimitiveType = iceberg_type
   ```
   Now we can also remove the `assert`'s to appease mypy.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1285260218


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1039,335 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.values()).union({"INT96"})
+
+
+class StatsAggregator:
+    current_min: Any
+    current_max: Any
+    trunc_length: Optional[int]
+
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min = None
+        self.current_max = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        expected_physical_type = _PRIMITIVE_TO_PHYSICAL[iceberg_type]
+        if expected_physical_type != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {expected_physical_type}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        self.current_min = val if self.current_min is None else min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        self.current_max = val if self.current_max is None else max(val, self.current_max)
+
+    def get_min(self) -> bytes:
+        return self.serialize(
+            self.current_min
+            if self.trunc_length is None
+            else TruncateTransform(width=self.trunc_length).transform(self.primitive_type)(self.current_min)
+        )
+
+    def get_max(self) -> Optional[bytes]:
+        if self.current_max is None:
+            return None
+
+        if self.primitive_type == StringType():
+            if type(self.current_max) != str:
+                raise ValueError("Expected the current_max to be a string")
+
+            s_result = self.current_max[: self.trunc_length]
+            if s_result != self.current_max:
+                chars = [*s_result]
+
+                for i in range(-1, -len(s_result) - 1, -1):
+                    try:
+                        to_inc = ord(chars[i])
+                        # will raise exception if the highest unicode code is reached
+                        _next = chr(to_inc + 1)
+                        chars[i] = _next
+                        return self.serialize("".join(chars))
+                    except ValueError:
+                        pass
+                return None  # didn't find a valid upper bound
+            return self.serialize(s_result)
+        elif self.primitive_type == BinaryType():
+            if type(self.current_max) != bytes:
+                raise ValueError("Expected the current_max to be bytes")
+            b_result = self.current_max[: self.trunc_length]
+            if b_result != self.current_max:
+                _bytes = [*b_result]
+                for i in range(-1, -len(b_result) - 1, -1):
+                    if _bytes[i] < 255:
+                        _bytes[i] += 1
+                        return b"".join([i.to_bytes(1, byteorder="little") for i in _bytes])
+                return None
+
+            return self.serialize(b_result)
+        else:
+            return self.serialize(self.current_max)[: self.trunc_length]
+
+
+DEFAULT_TRUNCATION_LENGTH = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+
+class MetricModeTypes(Enum):
+    TRUNCATE = "truncate"
+    NONE = "none"
+    COUNTS = "counts"
+    FULL = "full"
+
+
+DEFAULT_METRICS_MODE_KEY = "write.metadata.metrics.default"
+COLUMN_METRICS_MODE_KEY_PREFIX = "write.metadata.metrics.column"
+
+
+@dataclass(frozen=True)
+class MetricsMode(Singleton):
+    type: MetricModeTypes
+    length: Optional[int] = None
+
+
+def match_metrics_mode(mode: str) -> MetricsMode:
+    sanitized_mode = mode.lower()
+    if sanitized_mode.startswith("truncate"):
+        m = re.match(TRUNCATION_EXPR, mode, re.IGNORECASE)
+        if m:
+            length = int(m[1])
+            if length < 1:
+                raise ValueError("Truncation length must be larger than 0")
+            return MetricsMode(MetricModeTypes.TRUNCATE, int(m[1]))
+        else:
+            raise ValueError(f"Malformed truncate: {mode}")
+    elif sanitized_mode.startswith("none"):
+        return MetricsMode(MetricModeTypes.NONE)
+    elif sanitized_mode.startswith("counts"):
+        return MetricsMode(MetricModeTypes.COUNTS)
+    elif sanitized_mode.startswith("full"):
+        return MetricsMode(MetricModeTypes.FULL)
+    else:
+        raise ValueError(f"Unsupported metrics mode: {mode}")
+
+
+@dataclass(frozen=True)
+class StatisticsCollector:
+    field_id: int
+    iceberg_type: PrimitiveType
+    mode: MetricsMode
+    column_name: str
+
+
+class PyArrowStatisticsCollector(PreOrderSchemaVisitor[List[StatisticsCollector]]):
+    _field_id: int = 0
+    _schema: Schema
+    _properties: Dict[str, str]
+
+    def __init__(self, schema: Schema, properties: Dict[str, str]):
+        self._schema = schema
+        self._properties = properties
+
+    def schema(self, schema: Schema, struct_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        return struct_result()
+
+    def struct(
+        self, struct: StructType, field_results: List[Callable[[], List[StatisticsCollector]]]
+    ) -> List[StatisticsCollector]:
+        return list(chain(*[result() for result in field_results]))
+
+    def field(self, field: NestedField, field_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = field.field_id
+        result = field_result()
+        return result
+
+    def list(self, list_type: ListType, element_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = list_type.element_id
+        return element_result()
+
+    def map(
+        self,
+        map_type: MapType,
+        key_result: Callable[[], List[StatisticsCollector]],
+        value_result: Callable[[], List[StatisticsCollector]],
+    ) -> List[StatisticsCollector]:
+        self._field_id = map_type.key_id
+        k = key_result()
+        self._field_id = map_type.value_id
+        v = value_result()
+        return k + v
+
+    def primitive(self, primitive: PrimitiveType) -> List[StatisticsCollector]:
+        column_name = self._schema.find_column_name(self._field_id)
+        if column_name is None:
+            raise ValueError(f"Column for field {self._field_id} not found")
+
+        metrics_mode = MetricsMode(MetricModeTypes.TRUNCATE, DEFAULT_TRUNCATION_LENGTH)
+
+        default_mode = self._properties.get(DEFAULT_METRICS_MODE_KEY)
+        if default_mode:
+            metrics_mode = match_metrics_mode(default_mode)
+
+        col_mode = self._properties.get(f"{COLUMN_METRICS_MODE_KEY_PREFIX}.{column_name}")
+        if col_mode:
+            metrics_mode = match_metrics_mode(col_mode)
+
+        if (
+            not (isinstance(primitive, StringType) or isinstance(primitive, BinaryType))
+            and metrics_mode.type == MetricModeTypes.TRUNCATE
+        ):
+            metrics_mode = MetricsMode(MetricModeTypes.FULL)
+
+        return [StatisticsCollector(field_id=self._field_id, iceberg_type=primitive, mode=metrics_mode, column_name=column_name)]
+
+
+def fill_parquet_file_metadata(
+    df: DataFile,
+    parquet_metadata: pq.FileMetaData,
+    file_size: int,
+    table_metadata: TableMetadata,
+) -> None:
+    """
+    Computes and fills the following fields of the DataFile object.
+
+    - file_format
+    - record_count
+    - file_size_in_bytes
+    - column_sizes
+    - value_counts
+    - null_value_counts
+    - nan_value_counts
+    - lower_bounds
+    - upper_bounds
+    - split_offsets
+
+    Args:
+        df (DataFile): A DataFile object representing the Parquet file for which metadata is to be filled.
+        parquet_metadata (pyarrow.parquet.FileMetaData): A pyarrow metadata object.
+        file_size (int): The total compressed file size cannot be retrieved from the metadata and hence has to
+            be passed here. Depending on the kind of file system and pyarrow library call used, different
+            ways to obtain this value might be appropriate.
+        table_metadata (pyiceberg.table.metadata.TableMetadata): The Iceberg table metadata. It is required to
+            compute the mapping if column position to iceberg schema type id. It's also used to set the mode
+            for column metrics collection
+    """
+    schema = next(filter(lambda s: s.schema_id == table_metadata.current_schema_id, table_metadata.schemas))
+
+    stats_columns = pre_order_visit(schema, PyArrowStatisticsCollector(schema, table_metadata.properties))
+
+    if parquet_metadata.num_columns != len(stats_columns):
+        raise ValueError(
+            f"Number of columns in metadata ({len(stats_columns)}) is different from the number of columns in pyarrow table ({parquet_metadata.num_columns})"
+        )
+
+    column_sizes: Dict[int, int] = {}
+    value_counts: Dict[int, int] = {}
+    split_offsets: List[int] = []
+
+    null_value_counts: Dict[int, int] = {}
+    nan_value_counts: Dict[int, int] = {}
+
+    col_aggs = {}
+
+    for r in range(parquet_metadata.num_row_groups):
+        # References:
+        # https://github.com/apache/iceberg/blob/fc381a81a1fdb8f51a0637ca27cd30673bd7aad3/parquet/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java#L232
+        # https://github.com/apache/parquet-mr/blob/ac29db4611f86a07cc6877b416aa4b183e09b353/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/ColumnChunkMetaData.java#L184
+
+        row_group = parquet_metadata.row_group(r)
+
+        data_offset = row_group.column(0).data_page_offset
+        dictionary_offset = row_group.column(0).dictionary_page_offset
+
+        if row_group.column(0).has_dictionary_page and dictionary_offset < data_offset:
+            split_offsets.append(dictionary_offset)
+        else:
+            split_offsets.append(data_offset)
+
+        for pos, stats_col in enumerate(stats_columns):
+            field_id = stats_col.field_id
+
+            column = row_group.column(pos)

Review Comment:
   I don't think it is a good idea to match by position. There is no guarantee that the current schema of the table still matches the write schema used for this data file. Even making assumptions about how this method will be called, this is dangerous.
   
   Instead, I think this should index the columns by `path_in_schema`:
   
   ```python
   columns_by_path = { row_group.column(pos).path_in_schema: row_group.column(pos) for pos in range(row_group.num_columns) }
   ```
   
   Then this code can get the metrics for each column using `StatisticsCollector.column_name` to find the stats in the map.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1285260607


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1039,335 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.values()).union({"INT96"})
+
+
+class StatsAggregator:
+    current_min: Any
+    current_max: Any
+    trunc_length: Optional[int]
+
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min = None
+        self.current_max = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        expected_physical_type = _PRIMITIVE_TO_PHYSICAL[iceberg_type]
+        if expected_physical_type != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {expected_physical_type}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        self.current_min = val if self.current_min is None else min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        self.current_max = val if self.current_max is None else max(val, self.current_max)
+
+    def get_min(self) -> bytes:
+        return self.serialize(
+            self.current_min
+            if self.trunc_length is None
+            else TruncateTransform(width=self.trunc_length).transform(self.primitive_type)(self.current_min)
+        )
+
+    def get_max(self) -> Optional[bytes]:
+        if self.current_max is None:
+            return None
+
+        if self.primitive_type == StringType():
+            if type(self.current_max) != str:
+                raise ValueError("Expected the current_max to be a string")
+
+            s_result = self.current_max[: self.trunc_length]
+            if s_result != self.current_max:
+                chars = [*s_result]
+
+                for i in range(-1, -len(s_result) - 1, -1):
+                    try:
+                        to_inc = ord(chars[i])
+                        # will raise exception if the highest unicode code is reached
+                        _next = chr(to_inc + 1)
+                        chars[i] = _next
+                        return self.serialize("".join(chars))
+                    except ValueError:
+                        pass
+                return None  # didn't find a valid upper bound
+            return self.serialize(s_result)
+        elif self.primitive_type == BinaryType():
+            if type(self.current_max) != bytes:
+                raise ValueError("Expected the current_max to be bytes")
+            b_result = self.current_max[: self.trunc_length]
+            if b_result != self.current_max:
+                _bytes = [*b_result]
+                for i in range(-1, -len(b_result) - 1, -1):
+                    if _bytes[i] < 255:
+                        _bytes[i] += 1
+                        return b"".join([i.to_bytes(1, byteorder="little") for i in _bytes])
+                return None
+
+            return self.serialize(b_result)
+        else:
+            return self.serialize(self.current_max)[: self.trunc_length]
+
+
+DEFAULT_TRUNCATION_LENGTH = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+
+class MetricModeTypes(Enum):
+    TRUNCATE = "truncate"
+    NONE = "none"
+    COUNTS = "counts"
+    FULL = "full"
+
+
+DEFAULT_METRICS_MODE_KEY = "write.metadata.metrics.default"
+COLUMN_METRICS_MODE_KEY_PREFIX = "write.metadata.metrics.column"
+
+
+@dataclass(frozen=True)
+class MetricsMode(Singleton):
+    type: MetricModeTypes
+    length: Optional[int] = None
+
+
+def match_metrics_mode(mode: str) -> MetricsMode:
+    sanitized_mode = mode.lower()
+    if sanitized_mode.startswith("truncate"):
+        m = re.match(TRUNCATION_EXPR, mode, re.IGNORECASE)
+        if m:
+            length = int(m[1])
+            if length < 1:
+                raise ValueError("Truncation length must be larger than 0")
+            return MetricsMode(MetricModeTypes.TRUNCATE, int(m[1]))
+        else:
+            raise ValueError(f"Malformed truncate: {mode}")
+    elif sanitized_mode.startswith("none"):
+        return MetricsMode(MetricModeTypes.NONE)
+    elif sanitized_mode.startswith("counts"):
+        return MetricsMode(MetricModeTypes.COUNTS)
+    elif sanitized_mode.startswith("full"):
+        return MetricsMode(MetricModeTypes.FULL)
+    else:
+        raise ValueError(f"Unsupported metrics mode: {mode}")
+
+
+@dataclass(frozen=True)
+class StatisticsCollector:
+    field_id: int
+    iceberg_type: PrimitiveType
+    mode: MetricsMode
+    column_name: str
+
+
+class PyArrowStatisticsCollector(PreOrderSchemaVisitor[List[StatisticsCollector]]):
+    _field_id: int = 0
+    _schema: Schema
+    _properties: Dict[str, str]
+
+    def __init__(self, schema: Schema, properties: Dict[str, str]):
+        self._schema = schema
+        self._properties = properties
+
+    def schema(self, schema: Schema, struct_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        return struct_result()
+
+    def struct(
+        self, struct: StructType, field_results: List[Callable[[], List[StatisticsCollector]]]
+    ) -> List[StatisticsCollector]:
+        return list(chain(*[result() for result in field_results]))
+
+    def field(self, field: NestedField, field_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = field.field_id
+        result = field_result()
+        return result
+
+    def list(self, list_type: ListType, element_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = list_type.element_id
+        return element_result()
+
+    def map(
+        self,
+        map_type: MapType,
+        key_result: Callable[[], List[StatisticsCollector]],
+        value_result: Callable[[], List[StatisticsCollector]],
+    ) -> List[StatisticsCollector]:
+        self._field_id = map_type.key_id
+        k = key_result()
+        self._field_id = map_type.value_id
+        v = value_result()
+        return k + v
+
+    def primitive(self, primitive: PrimitiveType) -> List[StatisticsCollector]:
+        column_name = self._schema.find_column_name(self._field_id)
+        if column_name is None:
+            raise ValueError(f"Column for field {self._field_id} not found")
+
+        metrics_mode = MetricsMode(MetricModeTypes.TRUNCATE, DEFAULT_TRUNCATION_LENGTH)
+
+        default_mode = self._properties.get(DEFAULT_METRICS_MODE_KEY)
+        if default_mode:
+            metrics_mode = match_metrics_mode(default_mode)
+
+        col_mode = self._properties.get(f"{COLUMN_METRICS_MODE_KEY_PREFIX}.{column_name}")
+        if col_mode:
+            metrics_mode = match_metrics_mode(col_mode)
+
+        if (
+            not (isinstance(primitive, StringType) or isinstance(primitive, BinaryType))
+            and metrics_mode.type == MetricModeTypes.TRUNCATE
+        ):
+            metrics_mode = MetricsMode(MetricModeTypes.FULL)
+
+        return [StatisticsCollector(field_id=self._field_id, iceberg_type=primitive, mode=metrics_mode, column_name=column_name)]
+
+
+def fill_parquet_file_metadata(
+    df: DataFile,
+    parquet_metadata: pq.FileMetaData,
+    file_size: int,
+    table_metadata: TableMetadata,

Review Comment:
   I don't think that this should be passed in. All that's needed is the Parquet write schema and a map from field ID to metrics mode. There are good reasons not to require that bundled into `TableMetadata`:
   1. It is safe and possible to write new data files with any older schema, so the current table schema may not be the write schema (this would currently break in that case)
   2. Metrics modes can be pre-calculated to produce a map from field ID to mode. It makes more sense to do this in a common util method than to mix the code into the already complex `StatisticsCollector` visitor
   3. Avoiding passing `TableMetadata` keeps this logic cleaner and forces some separation of concerns



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1285261271


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1039,335 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.values()).union({"INT96"})
+
+
+class StatsAggregator:
+    current_min: Any
+    current_max: Any
+    trunc_length: Optional[int]
+
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min = None
+        self.current_max = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        expected_physical_type = _PRIMITIVE_TO_PHYSICAL[iceberg_type]
+        if expected_physical_type != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {expected_physical_type}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        self.current_min = val if self.current_min is None else min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        self.current_max = val if self.current_max is None else max(val, self.current_max)
+
+    def get_min(self) -> bytes:
+        return self.serialize(
+            self.current_min
+            if self.trunc_length is None
+            else TruncateTransform(width=self.trunc_length).transform(self.primitive_type)(self.current_min)
+        )
+
+    def get_max(self) -> Optional[bytes]:
+        if self.current_max is None:
+            return None
+
+        if self.primitive_type == StringType():
+            if type(self.current_max) != str:
+                raise ValueError("Expected the current_max to be a string")
+
+            s_result = self.current_max[: self.trunc_length]
+            if s_result != self.current_max:
+                chars = [*s_result]
+
+                for i in range(-1, -len(s_result) - 1, -1):
+                    try:
+                        to_inc = ord(chars[i])
+                        # will raise exception if the highest unicode code is reached
+                        _next = chr(to_inc + 1)
+                        chars[i] = _next
+                        return self.serialize("".join(chars))
+                    except ValueError:
+                        pass
+                return None  # didn't find a valid upper bound
+            return self.serialize(s_result)
+        elif self.primitive_type == BinaryType():
+            if type(self.current_max) != bytes:
+                raise ValueError("Expected the current_max to be bytes")
+            b_result = self.current_max[: self.trunc_length]
+            if b_result != self.current_max:
+                _bytes = [*b_result]
+                for i in range(-1, -len(b_result) - 1, -1):
+                    if _bytes[i] < 255:
+                        _bytes[i] += 1
+                        return b"".join([i.to_bytes(1, byteorder="little") for i in _bytes])
+                return None
+
+            return self.serialize(b_result)
+        else:
+            return self.serialize(self.current_max)[: self.trunc_length]
+
+
+DEFAULT_TRUNCATION_LENGTH = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+
+class MetricModeTypes(Enum):
+    TRUNCATE = "truncate"
+    NONE = "none"
+    COUNTS = "counts"
+    FULL = "full"
+
+
+DEFAULT_METRICS_MODE_KEY = "write.metadata.metrics.default"
+COLUMN_METRICS_MODE_KEY_PREFIX = "write.metadata.metrics.column"
+
+
+@dataclass(frozen=True)
+class MetricsMode(Singleton):
+    type: MetricModeTypes
+    length: Optional[int] = None
+
+
+def match_metrics_mode(mode: str) -> MetricsMode:
+    sanitized_mode = mode.lower()
+    if sanitized_mode.startswith("truncate"):
+        m = re.match(TRUNCATION_EXPR, mode, re.IGNORECASE)
+        if m:
+            length = int(m[1])
+            if length < 1:
+                raise ValueError("Truncation length must be larger than 0")
+            return MetricsMode(MetricModeTypes.TRUNCATE, int(m[1]))
+        else:
+            raise ValueError(f"Malformed truncate: {mode}")
+    elif sanitized_mode.startswith("none"):
+        return MetricsMode(MetricModeTypes.NONE)
+    elif sanitized_mode.startswith("counts"):
+        return MetricsMode(MetricModeTypes.COUNTS)
+    elif sanitized_mode.startswith("full"):
+        return MetricsMode(MetricModeTypes.FULL)
+    else:
+        raise ValueError(f"Unsupported metrics mode: {mode}")
+
+
+@dataclass(frozen=True)
+class StatisticsCollector:
+    field_id: int
+    iceberg_type: PrimitiveType
+    mode: MetricsMode
+    column_name: str
+
+
+class PyArrowStatisticsCollector(PreOrderSchemaVisitor[List[StatisticsCollector]]):
+    _field_id: int = 0
+    _schema: Schema
+    _properties: Dict[str, str]
+
+    def __init__(self, schema: Schema, properties: Dict[str, str]):
+        self._schema = schema
+        self._properties = properties
+
+    def schema(self, schema: Schema, struct_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        return struct_result()
+
+    def struct(
+        self, struct: StructType, field_results: List[Callable[[], List[StatisticsCollector]]]
+    ) -> List[StatisticsCollector]:
+        return list(chain(*[result() for result in field_results]))
+
+    def field(self, field: NestedField, field_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = field.field_id
+        result = field_result()
+        return result
+
+    def list(self, list_type: ListType, element_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = list_type.element_id
+        return element_result()
+
+    def map(
+        self,
+        map_type: MapType,
+        key_result: Callable[[], List[StatisticsCollector]],
+        value_result: Callable[[], List[StatisticsCollector]],
+    ) -> List[StatisticsCollector]:
+        self._field_id = map_type.key_id
+        k = key_result()
+        self._field_id = map_type.value_id
+        v = value_result()
+        return k + v
+
+    def primitive(self, primitive: PrimitiveType) -> List[StatisticsCollector]:
+        column_name = self._schema.find_column_name(self._field_id)
+        if column_name is None:
+            raise ValueError(f"Column for field {self._field_id} not found")
+
+        metrics_mode = MetricsMode(MetricModeTypes.TRUNCATE, DEFAULT_TRUNCATION_LENGTH)
+
+        default_mode = self._properties.get(DEFAULT_METRICS_MODE_KEY)
+        if default_mode:
+            metrics_mode = match_metrics_mode(default_mode)
+
+        col_mode = self._properties.get(f"{COLUMN_METRICS_MODE_KEY_PREFIX}.{column_name}")
+        if col_mode:
+            metrics_mode = match_metrics_mode(col_mode)
+
+        if (
+            not (isinstance(primitive, StringType) or isinstance(primitive, BinaryType))
+            and metrics_mode.type == MetricModeTypes.TRUNCATE
+        ):
+            metrics_mode = MetricsMode(MetricModeTypes.FULL)
+
+        return [StatisticsCollector(field_id=self._field_id, iceberg_type=primitive, mode=metrics_mode, column_name=column_name)]
+
+
+def fill_parquet_file_metadata(
+    df: DataFile,
+    parquet_metadata: pq.FileMetaData,
+    file_size: int,
+    table_metadata: TableMetadata,
+) -> None:
+    """
+    Computes and fills the following fields of the DataFile object.
+
+    - file_format
+    - record_count
+    - file_size_in_bytes
+    - column_sizes
+    - value_counts
+    - null_value_counts
+    - nan_value_counts
+    - lower_bounds
+    - upper_bounds
+    - split_offsets
+
+    Args:
+        df (DataFile): A DataFile object representing the Parquet file for which metadata is to be filled.
+        parquet_metadata (pyarrow.parquet.FileMetaData): A pyarrow metadata object.
+        file_size (int): The total compressed file size cannot be retrieved from the metadata and hence has to
+            be passed here. Depending on the kind of file system and pyarrow library call used, different
+            ways to obtain this value might be appropriate.
+        table_metadata (pyiceberg.table.metadata.TableMetadata): The Iceberg table metadata. It is required to
+            compute the mapping if column position to iceberg schema type id. It's also used to set the mode
+            for column metrics collection
+    """
+    schema = next(filter(lambda s: s.schema_id == table_metadata.current_schema_id, table_metadata.schemas))
+
+    stats_columns = pre_order_visit(schema, PyArrowStatisticsCollector(schema, table_metadata.properties))
+
+    if parquet_metadata.num_columns != len(stats_columns):
+        raise ValueError(
+            f"Number of columns in metadata ({len(stats_columns)}) is different from the number of columns in pyarrow table ({parquet_metadata.num_columns})"
+        )
+
+    column_sizes: Dict[int, int] = {}
+    value_counts: Dict[int, int] = {}
+    split_offsets: List[int] = []
+
+    null_value_counts: Dict[int, int] = {}
+    nan_value_counts: Dict[int, int] = {}
+
+    col_aggs = {}
+
+    for r in range(parquet_metadata.num_row_groups):
+        # References:
+        # https://github.com/apache/iceberg/blob/fc381a81a1fdb8f51a0637ca27cd30673bd7aad3/parquet/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java#L232
+        # https://github.com/apache/parquet-mr/blob/ac29db4611f86a07cc6877b416aa4b183e09b353/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/ColumnChunkMetaData.java#L184
+
+        row_group = parquet_metadata.row_group(r)
+
+        data_offset = row_group.column(0).data_page_offset
+        dictionary_offset = row_group.column(0).dictionary_page_offset
+
+        if row_group.column(0).has_dictionary_page and dictionary_offset < data_offset:
+            split_offsets.append(dictionary_offset)
+        else:
+            split_offsets.append(data_offset)
+
+        for pos, stats_col in enumerate(stats_columns):
+            field_id = stats_col.field_id
+
+            column = row_group.column(pos)
+
+            column_sizes[field_id] = column_sizes.get(field_id, 0) + column.total_compressed_size
+
+            if stats_col.mode == MetricsMode(MetricModeTypes.NONE):
+                continue
+
+            value_counts[field_id] = value_counts.get(field_id, 0) + column.num_values
+
+            if column.is_stats_set:
+                try:
+                    statistics = column.statistics
+
+                    null_value_counts[field_id] = null_value_counts.get(field_id, 0) + statistics.null_count
+
+                    if stats_col.mode == MetricsMode(MetricModeTypes.COUNTS):
+                        continue
+
+                    if field_id not in col_aggs:
+                        col_aggs[field_id] = StatsAggregator(
+                            stats_col.iceberg_type, statistics.physical_type, stats_col.mode.length
+                        )
+
+                    col_aggs[field_id].add_min(statistics.min)
+                    col_aggs[field_id].add_max(statistics.max)
+
+                except pyarrow.lib.ArrowNotImplementedError as e:
+                    logger.warning(e)
+            else:
+                logger.warning("PyArrow statistics missing for column %d when writing file", pos)
+
+    split_offsets.sort()
+
+    lower_bounds = {}
+    upper_bounds = {}
+
+    for k, agg in col_aggs.items():
+        lower_bounds[k] = agg.get_min()

Review Comment:
   Min could also be None, in which case it should not be written.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1285262494


##########
python/tests/io/test_pyarrow.py:
##########
@@ -1345,3 +1374,655 @@ def test_pyarrow_wrap_fsspec(example_task: FileScanTask, table_schema_simple: Sc
 bar: [[1,2,3]]
 baz: [[true,false,null]]"""
     )
+
+
+def construct_test_table() -> Tuple[Any, Any, Union[TableMetadataV1, TableMetadataV2]]:
+    table_metadata = {
+        "format-version": 2,
+        "location": "s3://bucket/test/location",
+        "last-column-id": 7,
+        "current-schema-id": 0,
+        "schemas": [
+            {
+                "type": "struct",
+                "schema-id": 0,
+                "fields": [
+                    {"id": 1, "name": "strings", "required": False, "type": "string"},
+                    {"id": 2, "name": "floats", "required": False, "type": "float"},
+                    {
+                        "id": 3,
+                        "name": "list",
+                        "required": False,
+                        "type": {"type": "list", "element-id": 5, "element": "long", "element-required": False},
+                    },
+                    {
+                        "id": 4,
+                        "name": "maps",
+                        "required": False,
+                        "type": {
+                            "type": "map",
+                            "key-id": 6,
+                            "key": "long",
+                            "value-id": 7,
+                            "value": "long",
+                            "value-required": False,
+                        },
+                    },
+                ],
+            },
+        ],
+        "default-spec-id": 0,
+        "partition-specs": [{"spec-id": 0, "fields": []}],
+        "properties": {},
+    }
+
+    table_metadata = TableMetadataUtil.parse_obj(table_metadata)
+    arrow_schema = schema_to_pyarrow(table_metadata.schemas[0])
+
+    _strings = ["zzzzzzzzzzzzzzzzzzzz", "rrrrrrrrrrrrrrrrrrrr", None, "aaaaaaaaaaaaaaaaaaaa"]
+
+    _floats = [3.14, math.nan, 1.69, 100]
+
+    _list = [[1, 2, 3], [4, 5, 6], None, [7, 8, 9]]
+
+    _maps: List[Optional[Dict[int, int]]] = [
+        {1: 2, 3: 4},
+        None,
+        {5: 6},
+        {},
+    ]
+
+    table = pa.Table.from_pydict(
+        {
+            "strings": _strings,
+            "floats": _floats,
+            "list": _list,
+            "maps": _maps,
+        },
+        schema=arrow_schema,
+    )
+    metadata_collector: List[Any] = []
+
+    with pa.BufferOutputStream() as f:
+        with pq.ParquetWriter(f, table.schema, metadata_collector=metadata_collector) as writer:
+            writer.write_table(table)
+
+        return f.getvalue(), metadata_collector[0], table_metadata
+
+
+def test_record_count() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.record_count == 4
+
+
+def test_file_size() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.file_size_in_bytes == len(file_bytes)
+
+
+def test_value_counts() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.value_counts) == 5
+    assert datafile.value_counts[1] == 4
+    assert datafile.value_counts[2] == 4
+    assert datafile.value_counts[5] == 10  # 3 lists with 3 items and a None value
+    assert datafile.value_counts[6] == 5
+    assert datafile.value_counts[7] == 5
+
+
+def test_column_sizes() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.column_sizes) == 5
+    # these values are an artifact of how the write_table encodes the columns
+    assert datafile.column_sizes[1] == 116
+    assert datafile.column_sizes[2] == 89
+    assert datafile.column_sizes[5] == 151
+    assert datafile.column_sizes[6] == 117
+    assert datafile.column_sizes[7] == 117
+
+
+def test_null_and_nan_counts() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.null_value_counts) == 5
+    assert datafile.null_value_counts[1] == 1
+    assert datafile.null_value_counts[2] == 0
+    assert datafile.null_value_counts[5] == 1
+    assert datafile.null_value_counts[6] == 2
+    assert datafile.null_value_counts[7] == 2
+
+    # #arrow does not include this in the statistics
+    # assert len(datafile.nan_value_counts)  == 3
+    # assert datafile.nan_value_counts[1]    == 0

Review Comment:
   These should be `None` for any column that is not a float or double type.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1285262390


##########
python/tests/io/test_pyarrow.py:
##########
@@ -1345,3 +1374,655 @@ def test_pyarrow_wrap_fsspec(example_task: FileScanTask, table_schema_simple: Sc
 bar: [[1,2,3]]
 baz: [[true,false,null]]"""
     )
+
+
+def construct_test_table() -> Tuple[Any, Any, Union[TableMetadataV1, TableMetadataV2]]:
+    table_metadata = {
+        "format-version": 2,
+        "location": "s3://bucket/test/location",
+        "last-column-id": 7,
+        "current-schema-id": 0,
+        "schemas": [
+            {
+                "type": "struct",
+                "schema-id": 0,
+                "fields": [
+                    {"id": 1, "name": "strings", "required": False, "type": "string"},
+                    {"id": 2, "name": "floats", "required": False, "type": "float"},
+                    {
+                        "id": 3,
+                        "name": "list",
+                        "required": False,
+                        "type": {"type": "list", "element-id": 5, "element": "long", "element-required": False},
+                    },
+                    {
+                        "id": 4,
+                        "name": "maps",
+                        "required": False,
+                        "type": {
+                            "type": "map",
+                            "key-id": 6,
+                            "key": "long",
+                            "value-id": 7,
+                            "value": "long",
+                            "value-required": False,
+                        },
+                    },
+                ],
+            },
+        ],
+        "default-spec-id": 0,
+        "partition-specs": [{"spec-id": 0, "fields": []}],
+        "properties": {},
+    }
+
+    table_metadata = TableMetadataUtil.parse_obj(table_metadata)
+    arrow_schema = schema_to_pyarrow(table_metadata.schemas[0])
+
+    _strings = ["zzzzzzzzzzzzzzzzzzzz", "rrrrrrrrrrrrrrrrrrrr", None, "aaaaaaaaaaaaaaaaaaaa"]
+
+    _floats = [3.14, math.nan, 1.69, 100]
+
+    _list = [[1, 2, 3], [4, 5, 6], None, [7, 8, 9]]
+
+    _maps: List[Optional[Dict[int, int]]] = [
+        {1: 2, 3: 4},
+        None,
+        {5: 6},
+        {},
+    ]
+
+    table = pa.Table.from_pydict(
+        {
+            "strings": _strings,
+            "floats": _floats,
+            "list": _list,
+            "maps": _maps,
+        },
+        schema=arrow_schema,
+    )
+    metadata_collector: List[Any] = []
+
+    with pa.BufferOutputStream() as f:
+        with pq.ParquetWriter(f, table.schema, metadata_collector=metadata_collector) as writer:
+            writer.write_table(table)
+
+        return f.getvalue(), metadata_collector[0], table_metadata
+
+
+def test_record_count() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.record_count == 4
+
+
+def test_file_size() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.file_size_in_bytes == len(file_bytes)
+
+
+def test_value_counts() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.value_counts) == 5
+    assert datafile.value_counts[1] == 4
+    assert datafile.value_counts[2] == 4
+    assert datafile.value_counts[5] == 10  # 3 lists with 3 items and a None value
+    assert datafile.value_counts[6] == 5

Review Comment:
   It looks like both maps and lists count empty/null parent structures. I would expect this to be the number of leaf values.
   
   Can you confirm that this is also the behavior of the Java Parquet library? If not, we should just drop map and list stats because they aren't the same.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] maxdebayser commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "maxdebayser (via GitHub)" <gi...@apache.org>.
maxdebayser commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1288977036


##########
python/tests/io/test_pyarrow.py:
##########
@@ -1345,3 +1374,655 @@ def test_pyarrow_wrap_fsspec(example_task: FileScanTask, table_schema_simple: Sc
 bar: [[1,2,3]]
 baz: [[true,false,null]]"""
     )
+
+
+def construct_test_table() -> Tuple[Any, Any, Union[TableMetadataV1, TableMetadataV2]]:
+    table_metadata = {
+        "format-version": 2,
+        "location": "s3://bucket/test/location",
+        "last-column-id": 7,
+        "current-schema-id": 0,
+        "schemas": [
+            {
+                "type": "struct",
+                "schema-id": 0,
+                "fields": [
+                    {"id": 1, "name": "strings", "required": False, "type": "string"},
+                    {"id": 2, "name": "floats", "required": False, "type": "float"},
+                    {
+                        "id": 3,
+                        "name": "list",
+                        "required": False,
+                        "type": {"type": "list", "element-id": 5, "element": "long", "element-required": False},
+                    },
+                    {
+                        "id": 4,
+                        "name": "maps",
+                        "required": False,
+                        "type": {
+                            "type": "map",
+                            "key-id": 6,
+                            "key": "long",
+                            "value-id": 7,
+                            "value": "long",
+                            "value-required": False,
+                        },
+                    },
+                ],
+            },
+        ],
+        "default-spec-id": 0,
+        "partition-specs": [{"spec-id": 0, "fields": []}],
+        "properties": {},
+    }
+
+    table_metadata = TableMetadataUtil.parse_obj(table_metadata)
+    arrow_schema = schema_to_pyarrow(table_metadata.schemas[0])
+
+    _strings = ["zzzzzzzzzzzzzzzzzzzz", "rrrrrrrrrrrrrrrrrrrr", None, "aaaaaaaaaaaaaaaaaaaa"]
+
+    _floats = [3.14, math.nan, 1.69, 100]
+
+    _list = [[1, 2, 3], [4, 5, 6], None, [7, 8, 9]]
+
+    _maps: List[Optional[Dict[int, int]]] = [
+        {1: 2, 3: 4},
+        None,
+        {5: 6},
+        {},
+    ]
+
+    table = pa.Table.from_pydict(
+        {
+            "strings": _strings,
+            "floats": _floats,
+            "list": _list,
+            "maps": _maps,
+        },
+        schema=arrow_schema,
+    )
+    metadata_collector: List[Any] = []
+
+    with pa.BufferOutputStream() as f:
+        with pq.ParquetWriter(f, table.schema, metadata_collector=metadata_collector) as writer:
+            writer.write_table(table)
+
+        return f.getvalue(), metadata_collector[0], table_metadata
+
+
+def test_record_count() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.record_count == 4
+
+
+def test_file_size() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.file_size_in_bytes == len(file_bytes)
+
+
+def test_value_counts() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.value_counts) == 5
+    assert datafile.value_counts[1] == 4
+    assert datafile.value_counts[2] == 4
+    assert datafile.value_counts[5] == 10  # 3 lists with 3 items and a None value
+    assert datafile.value_counts[6] == 5

Review Comment:
   Yes, but the counts are computed. Can you test in Java with a list that contains Nulls and a Null list?



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1264727823


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1013,3 +1025,271 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+class StatsAggregator:
+    def __init__(self, type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.trunc_length = trunc_length
+        self.primitive_type: Optional[PrimitiveType] = None
+
+        if type_string == "BOOLEAN":
+            self.primitive_type = BooleanType()
+        elif type_string == "INT32":
+            self.primitive_type = IntegerType()
+        elif type_string == "INT64":
+            self.primitive_type = LongType()
+        elif type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+        elif type_string == "FLOAT":
+            self.primitive_type = FloatType()
+        elif type_string == "DOUBLE":
+            self.primitive_type = DoubleType()
+        elif type_string == "BYTE_ARRAY":
+            self.primitive_type = BinaryType()
+        elif type_string == "FIXED_LEN_BYTE_ARRAY":
+            self.primitive_type = BinaryType()
+        else:
+            raise AssertionError(f"Unknown physical type {type_string}")
+
+    def serialize(self, value: Any) -> bytes:
+        if type(value) == str:
+            value = value.encode()
+        assert self.primitive_type is not None  # appease mypy
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        if self.current_min is None:
+            self.current_min = val
+        else:
+            self.current_min = min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        if self.current_max is None:
+            self.current_max = val
+        else:
+            self.current_max = max(self.current_max, val)
+
+    def get_min(self) -> bytes:
+        return self.serialize(self.current_min)[: self.trunc_length]
+
+    def get_max(self) -> bytes:
+        return self.serialize(self.current_max)[: self.trunc_length]
+
+
+DEFAULT_TRUNCATION_LENGHT = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+
+class MetricModeTypes(Enum):
+    TRUNCATE = "truncate"
+    NONE = "none"
+    COUNTS = "counts"
+    FULL = "full"
+
+
+DEFAULT_METRICS_MODE_KEY = "write.metadata.metrics.default"
+COLUMN_METRICS_MODE_KEY = "write.metadata.metrics.column"
+
+
+@dataclass(frozen=True)
+class MetricsMode(Singleton):
+    type: MetricModeTypes
+    length: Optional[int] = None
+
+
+def match_metrics_mode(mode: str) -> MetricsMode:
+    m = re.match(TRUNCATION_EXPR, mode, re.IGNORECASE)
+    if m:
+        length = int(m[1])
+        if length < 1:
+            raise AssertionError("Truncation length must be larger than 0")
+        return MetricsMode(MetricModeTypes.TRUNCATE, int(m[1]))
+    elif re.match("^none$", mode, re.IGNORECASE):
+        return MetricsMode(MetricModeTypes.NONE)
+    elif re.match("^counts$", mode, re.IGNORECASE):
+        return MetricsMode(MetricModeTypes.COUNTS)
+    elif re.match("^full$", mode, re.IGNORECASE):
+        return MetricsMode(MetricModeTypes.FULL)
+    else:
+        raise AssertionError(f"Unsupported metrics mode {mode}")
+
+
+@dataclass(frozen=True)
+class StatisticsCollector:
+    field_id: int
+    iceberg_type: PrimitiveType
+    mode: MetricsMode
+    column_name: str
+
+
+class PyArrowStatisticsCollector(PreOrderSchemaVisitor[List[StatisticsCollector]]):
+    _field_id = 0
+    _schema: Schema
+    _properties: Dict[str, str]
+
+    def __init__(self, schema: Schema, properties: Dict[str, str]):
+        self._schema = schema
+        self._properties = properties
+
+    def schema(self, schema: Schema, struct_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        return struct_result()
+
+    def struct(
+        self, struct: StructType, field_results: List[Callable[[], List[StatisticsCollector]]]
+    ) -> List[StatisticsCollector]:
+        return list(chain(*[result() for result in field_results]))
+
+    def field(self, field: NestedField, field_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = field.field_id
+        result = field_result()
+        return result
+
+    def list(self, list_type: ListType, element_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = list_type.element_id
+        return element_result()
+
+    def map(
+        self,
+        map_type: MapType,
+        key_result: Callable[[], List[StatisticsCollector]],
+        value_result: Callable[[], List[StatisticsCollector]],
+    ) -> List[StatisticsCollector]:
+        self._field_id = map_type.key_id
+        k = key_result()
+        self._field_id = map_type.value_id
+        v = value_result()
+        return k + v
+
+    def primitive(self, primitive: PrimitiveType) -> List[StatisticsCollector]:
+        column_name = self._schema.find_column_name(self._field_id)
+        assert column_name is not None, f"Column for field {self._field_id} not found"
+
+        metrics_mode = MetricsMode(MetricModeTypes.TRUNCATE, DEFAULT_TRUNCATION_LENGHT)
+
+        default_mode = self._properties.get(DEFAULT_METRICS_MODE_KEY)
+        if default_mode:
+            metrics_mode = match_metrics_mode(default_mode)
+
+        col_mode = self._properties.get(f"{COLUMN_METRICS_MODE_KEY}.{column_name}")
+        if col_mode:
+            metrics_mode = match_metrics_mode(col_mode)
+
+        return [StatisticsCollector(field_id=self._field_id, iceberg_type=primitive, mode=metrics_mode, column_name=column_name)]
+
+
+def fill_parquet_file_metadata(
+    df: DataFile,
+    parquet_metadata: pq.FileMetaData,
+    file_size: int,
+    table_metadata: TableMetadata,
+) -> None:
+    """
+    Computes and fills the following fields of the DataFile object.
+
+    - file_format
+    - record_count
+    - file_size_in_bytes
+    - column_sizes
+    - value_counts
+    - null_value_counts
+    - nan_value_counts
+    - lower_bounds
+    - upper_bounds
+    - split_offsets
+
+    Args:
+        df (DataFile): A DataFile object representing the Parquet file for which metadata is to be filled.
+        parquet_metadata (pyarrow.parquet.FileMetaData): A pyarrow metadata object.
+        file_size (int): The total compressed file size cannot be retrieved from the metadata and hence has to
+            be passed here. Depending on the kind of file system and pyarrow library call used, different
+            ways to obtain this value might be appropriate.
+        table_metadata (pyiceberg.table.metadata.TableMetadata): The Iceberg table metadata. It is required to
+            compute the mapping if column position to iceberg schema type id. It's also used to set the mode
+            for column metrics collection
+    """
+    schema = next(filter(lambda s: s.schema_id == table_metadata.current_schema_id, table_metadata.schemas))
+
+    stats_columns = pre_order_visit(schema, PyArrowStatisticsCollector(schema, table_metadata.properties))
+    assert parquet_metadata.num_columns == len(
+        stats_columns
+    ), f"Number of columns in metadata ({len(stats_columns)}) is different from the number of columns in pyarrow table ({parquet_metadata.num_columns})"
+
+    column_sizes: Dict[int, int] = {}
+    value_counts: Dict[int, int] = {}
+    split_offsets: List[int] = []
+
+    null_value_counts: Dict[int, int] = {}
+    nan_value_counts: Dict[int, int] = {}
+
+    col_aggs = {}
+
+    for r in range(parquet_metadata.num_row_groups):
+        # References:
+        # https://github.com/apache/iceberg/blob/fc381a81a1fdb8f51a0637ca27cd30673bd7aad3/parquet/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java#L232
+        # https://github.com/apache/parquet-mr/blob/ac29db4611f86a07cc6877b416aa4b183e09b353/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/ColumnChunkMetaData.java#L184
+
+        row_group = parquet_metadata.row_group(r)
+
+        data_offset = row_group.column(0).data_page_offset
+        dictionary_offset = row_group.column(0).dictionary_page_offset
+
+        if row_group.column(0).has_dictionary_page and dictionary_offset < data_offset:
+            split_offsets.append(dictionary_offset)
+        else:
+            split_offsets.append(data_offset)
+
+        for c in range(parquet_metadata.num_columns):
+            col_id = stats_columns[c].field_id
+
+            column = row_group.column(c)
+
+            column_sizes[col_id] = column_sizes.get(col_id, 0) + column.total_compressed_size
+
+            metrics_mode = stats_columns[c].mode
+
+            if metrics_mode == MetricsMode(MetricModeTypes.NONE):
+                continue
+
+            value_counts[col_id] = value_counts.get(col_id, 0) + column.num_values
+
+            if column.is_stats_set:
+                try:
+                    statistics = column.statistics
+
+                    null_value_counts[col_id] = null_value_counts.get(col_id, 0) + statistics.null_count
+
+                    if metrics_mode == MetricsMode(MetricModeTypes.COUNTS):
+                        continue
+
+                    if col_id not in col_aggs:
+                        col_aggs[col_id] = StatsAggregator(statistics.physical_type, metrics_mode.length)
+
+                    col_aggs[col_id].add_min(statistics.min)
+                    col_aggs[col_id].add_max(statistics.max)

Review Comment:
   What do you think of the following? This way we have fewer lookups. Also, I think it is best to stick with the Iceberg naming (field-id over column-id).
   ```suggestion
           for pos, stats_col in enumerate(stats_columns):
               field_id = stats_col.field_id
   
               column = row_group.column(pos)
   
               column_sizes[field_id] = column_sizes.get(field_id, 0) + column.total_compressed_size
   
               if stats_col.mode == MetricsMode(MetricModeTypes.NONE):
                   continue
   
               value_counts[field_id] = value_counts.get(field_id, 0) + column.num_values
   
               if column.is_stats_set:
                   try:
                       statistics = column.statistics
   
                       null_value_counts[field_id] = null_value_counts.get(field_id, 0) + statistics.null_count
   
                       if stats_col.mode == MetricsMode(MetricModeTypes.COUNTS):
                           continue
   
                       if field_id not in col_aggs:
                           col_aggs[field_id] = StatsAggregator(statistics.physical_type, stats_col.mode.length)
   
                       col_aggs[field_id].add_min(statistics.min)
                       col_aggs[field_id].add_max(statistics.max)
   ```



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1263156428


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1013,3 +1025,271 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+class StatsAggregator:
+    def __init__(self, type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.trunc_length = trunc_length
+        self.primitive_type: Optional[PrimitiveType] = None
+
+        if type_string == "BOOLEAN":
+            self.primitive_type = BooleanType()
+        elif type_string == "INT32":
+            self.primitive_type = IntegerType()
+        elif type_string == "INT64":
+            self.primitive_type = LongType()
+        elif type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+        elif type_string == "FLOAT":
+            self.primitive_type = FloatType()
+        elif type_string == "DOUBLE":
+            self.primitive_type = DoubleType()
+        elif type_string == "BYTE_ARRAY":
+            self.primitive_type = BinaryType()
+        elif type_string == "FIXED_LEN_BYTE_ARRAY":
+            self.primitive_type = BinaryType()
+        else:
+            raise AssertionError(f"Unknown physical type {type_string}")
+
+    def serialize(self, value: Any) -> bytes:
+        if type(value) == str:
+            value = value.encode()
+        assert self.primitive_type is not None  # appease mypy
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        if self.current_min is None:
+            self.current_min = val
+        else:
+            self.current_min = min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        if self.current_max is None:
+            self.current_max = val
+        else:
+            self.current_max = max(self.current_max, val)
+
+    def get_min(self) -> bytes:
+        return self.serialize(self.current_min)[: self.trunc_length]
+
+    def get_max(self) -> bytes:
+        return self.serialize(self.current_max)[: self.trunc_length]
+
+
+DEFAULT_TRUNCATION_LENGHT = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+
+class MetricModeTypes(Enum):
+    TRUNCATE = "truncate"
+    NONE = "none"
+    COUNTS = "counts"
+    FULL = "full"
+
+
+DEFAULT_METRICS_MODE_KEY = "write.metadata.metrics.default"
+COLUMN_METRICS_MODE_KEY = "write.metadata.metrics.column"

Review Comment:
   This is a prefix, right?



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1281069330


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1040,291 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHISICAL_TYPES = ["BOOLEAN", "INT32", "INT64", "INT96", "FLOAT", "DOUBLE", "BYTE_ARRAY", "FIXED_LEN_BYTE_ARRAY"]
+
+
+class StatsAggregator:
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.trunc_length = trunc_length
+
+        assert physical_type_string in _PHISICAL_TYPES, f"Unknown physical type {physical_type_string}"
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+        assert (
+            _PRIMITIVE_TO_PHYSICAL[iceberg_type] == physical_type_string
+        ), f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {_PRIMITIVE_TO_PHYSICAL[iceberg_type]}"
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if type(value) == date:

Review Comment:
   Yes, let's refactor in a separate PR. 



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1285259555


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1039,335 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.values()).union({"INT96"})
+
+
+class StatsAggregator:
+    current_min: Any
+    current_max: Any
+    trunc_length: Optional[int]
+
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min = None
+        self.current_max = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        expected_physical_type = _PRIMITIVE_TO_PHYSICAL[iceberg_type]
+        if expected_physical_type != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {expected_physical_type}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        self.current_min = val if self.current_min is None else min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        self.current_max = val if self.current_max is None else max(val, self.current_max)
+
+    def get_min(self) -> bytes:
+        return self.serialize(
+            self.current_min
+            if self.trunc_length is None
+            else TruncateTransform(width=self.trunc_length).transform(self.primitive_type)(self.current_min)
+        )
+
+    def get_max(self) -> Optional[bytes]:
+        if self.current_max is None:
+            return None
+
+        if self.primitive_type == StringType():
+            if type(self.current_max) != str:
+                raise ValueError("Expected the current_max to be a string")
+
+            s_result = self.current_max[: self.trunc_length]
+            if s_result != self.current_max:
+                chars = [*s_result]
+
+                for i in range(-1, -len(s_result) - 1, -1):
+                    try:
+                        to_inc = ord(chars[i])
+                        # will raise exception if the highest unicode code is reached
+                        _next = chr(to_inc + 1)
+                        chars[i] = _next
+                        return self.serialize("".join(chars))
+                    except ValueError:
+                        pass
+                return None  # didn't find a valid upper bound
+            return self.serialize(s_result)
+        elif self.primitive_type == BinaryType():
+            if type(self.current_max) != bytes:
+                raise ValueError("Expected the current_max to be bytes")
+            b_result = self.current_max[: self.trunc_length]
+            if b_result != self.current_max:
+                _bytes = [*b_result]
+                for i in range(-1, -len(b_result) - 1, -1):
+                    if _bytes[i] < 255:
+                        _bytes[i] += 1
+                        return b"".join([i.to_bytes(1, byteorder="little") for i in _bytes])
+                return None
+
+            return self.serialize(b_result)
+        else:
+            return self.serialize(self.current_max)[: self.trunc_length]
+
+
+DEFAULT_TRUNCATION_LENGTH = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+
+class MetricModeTypes(Enum):
+    TRUNCATE = "truncate"
+    NONE = "none"
+    COUNTS = "counts"
+    FULL = "full"
+
+
+DEFAULT_METRICS_MODE_KEY = "write.metadata.metrics.default"
+COLUMN_METRICS_MODE_KEY_PREFIX = "write.metadata.metrics.column"
+
+
+@dataclass(frozen=True)
+class MetricsMode(Singleton):
+    type: MetricModeTypes
+    length: Optional[int] = None
+
+
+def match_metrics_mode(mode: str) -> MetricsMode:
+    sanitized_mode = mode.lower()
+    if sanitized_mode.startswith("truncate"):
+        m = re.match(TRUNCATION_EXPR, mode, re.IGNORECASE)
+        if m:
+            length = int(m[1])
+            if length < 1:
+                raise ValueError("Truncation length must be larger than 0")
+            return MetricsMode(MetricModeTypes.TRUNCATE, int(m[1]))
+        else:
+            raise ValueError(f"Malformed truncate: {mode}")
+    elif sanitized_mode.startswith("none"):
+        return MetricsMode(MetricModeTypes.NONE)
+    elif sanitized_mode.startswith("counts"):
+        return MetricsMode(MetricModeTypes.COUNTS)
+    elif sanitized_mode.startswith("full"):
+        return MetricsMode(MetricModeTypes.FULL)
+    else:
+        raise ValueError(f"Unsupported metrics mode: {mode}")
+
+
+@dataclass(frozen=True)
+class StatisticsCollector:
+    field_id: int
+    iceberg_type: PrimitiveType
+    mode: MetricsMode
+    column_name: str
+
+
+class PyArrowStatisticsCollector(PreOrderSchemaVisitor[List[StatisticsCollector]]):
+    _field_id: int = 0
+    _schema: Schema
+    _properties: Dict[str, str]
+
+    def __init__(self, schema: Schema, properties: Dict[str, str]):
+        self._schema = schema
+        self._properties = properties
+
+    def schema(self, schema: Schema, struct_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        return struct_result()
+
+    def struct(
+        self, struct: StructType, field_results: List[Callable[[], List[StatisticsCollector]]]
+    ) -> List[StatisticsCollector]:
+        return list(chain(*[result() for result in field_results]))
+
+    def field(self, field: NestedField, field_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = field.field_id
+        result = field_result()
+        return result
+
+    def list(self, list_type: ListType, element_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = list_type.element_id
+        return element_result()
+
+    def map(
+        self,
+        map_type: MapType,
+        key_result: Callable[[], List[StatisticsCollector]],
+        value_result: Callable[[], List[StatisticsCollector]],
+    ) -> List[StatisticsCollector]:
+        self._field_id = map_type.key_id
+        k = key_result()
+        self._field_id = map_type.value_id
+        v = value_result()
+        return k + v
+
+    def primitive(self, primitive: PrimitiveType) -> List[StatisticsCollector]:
+        column_name = self._schema.find_column_name(self._field_id)
+        if column_name is None:
+            raise ValueError(f"Column for field {self._field_id} not found")
+
+        metrics_mode = MetricsMode(MetricModeTypes.TRUNCATE, DEFAULT_TRUNCATION_LENGTH)
+
+        default_mode = self._properties.get(DEFAULT_METRICS_MODE_KEY)
+        if default_mode:
+            metrics_mode = match_metrics_mode(default_mode)
+
+        col_mode = self._properties.get(f"{COLUMN_METRICS_MODE_KEY_PREFIX}.{column_name}")
+        if col_mode:
+            metrics_mode = match_metrics_mode(col_mode)
+
+        if (
+            not (isinstance(primitive, StringType) or isinstance(primitive, BinaryType))
+            and metrics_mode.type == MetricModeTypes.TRUNCATE
+        ):
+            metrics_mode = MetricsMode(MetricModeTypes.FULL)
+
+        return [StatisticsCollector(field_id=self._field_id, iceberg_type=primitive, mode=metrics_mode, column_name=column_name)]
+
+
+def fill_parquet_file_metadata(
+    df: DataFile,
+    parquet_metadata: pq.FileMetaData,
+    file_size: int,
+    table_metadata: TableMetadata,
+) -> None:
+    """
+    Computes and fills the following fields of the DataFile object.
+
+    - file_format
+    - record_count
+    - file_size_in_bytes
+    - column_sizes
+    - value_counts
+    - null_value_counts
+    - nan_value_counts
+    - lower_bounds
+    - upper_bounds
+    - split_offsets
+
+    Args:
+        df (DataFile): A DataFile object representing the Parquet file for which metadata is to be filled.
+        parquet_metadata (pyarrow.parquet.FileMetaData): A pyarrow metadata object.
+        file_size (int): The total compressed file size cannot be retrieved from the metadata and hence has to
+            be passed here. Depending on the kind of file system and pyarrow library call used, different
+            ways to obtain this value might be appropriate.
+        table_metadata (pyiceberg.table.metadata.TableMetadata): The Iceberg table metadata. It is required to
+            compute the mapping if column position to iceberg schema type id. It's also used to set the mode
+            for column metrics collection
+    """
+    schema = next(filter(lambda s: s.schema_id == table_metadata.current_schema_id, table_metadata.schemas))
+
+    stats_columns = pre_order_visit(schema, PyArrowStatisticsCollector(schema, table_metadata.properties))
+
+    if parquet_metadata.num_columns != len(stats_columns):

Review Comment:
   I think it should be fine to omit some metadata if it can't be calculated. I'd rather have metadata for some columns than only for all columns.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1285258312


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1039,335 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.values()).union({"INT96"})
+
+
+class StatsAggregator:
+    current_min: Any
+    current_max: Any
+    trunc_length: Optional[int]
+
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min = None
+        self.current_max = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        expected_physical_type = _PRIMITIVE_TO_PHYSICAL[iceberg_type]
+        if expected_physical_type != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {expected_physical_type}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        self.current_min = val if self.current_min is None else min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        self.current_max = val if self.current_max is None else max(val, self.current_max)
+
+    def get_min(self) -> bytes:
+        return self.serialize(
+            self.current_min
+            if self.trunc_length is None
+            else TruncateTransform(width=self.trunc_length).transform(self.primitive_type)(self.current_min)
+        )
+
+    def get_max(self) -> Optional[bytes]:
+        if self.current_max is None:
+            return None
+
+        if self.primitive_type == StringType():
+            if type(self.current_max) != str:
+                raise ValueError("Expected the current_max to be a string")
+
+            s_result = self.current_max[: self.trunc_length]
+            if s_result != self.current_max:
+                chars = [*s_result]
+
+                for i in range(-1, -len(s_result) - 1, -1):
+                    try:
+                        to_inc = ord(chars[i])
+                        # will raise exception if the highest unicode code is reached
+                        _next = chr(to_inc + 1)
+                        chars[i] = _next
+                        return self.serialize("".join(chars))
+                    except ValueError:
+                        pass
+                return None  # didn't find a valid upper bound
+            return self.serialize(s_result)
+        elif self.primitive_type == BinaryType():
+            if type(self.current_max) != bytes:
+                raise ValueError("Expected the current_max to be bytes")
+            b_result = self.current_max[: self.trunc_length]
+            if b_result != self.current_max:
+                _bytes = [*b_result]
+                for i in range(-1, -len(b_result) - 1, -1):
+                    if _bytes[i] < 255:
+                        _bytes[i] += 1
+                        return b"".join([i.to_bytes(1, byteorder="little") for i in _bytes])
+                return None
+
+            return self.serialize(b_result)
+        else:
+            return self.serialize(self.current_max)[: self.trunc_length]
+
+
+DEFAULT_TRUNCATION_LENGTH = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+
+class MetricModeTypes(Enum):
+    TRUNCATE = "truncate"
+    NONE = "none"
+    COUNTS = "counts"
+    FULL = "full"
+
+
+DEFAULT_METRICS_MODE_KEY = "write.metadata.metrics.default"
+COLUMN_METRICS_MODE_KEY_PREFIX = "write.metadata.metrics.column"
+
+
+@dataclass(frozen=True)
+class MetricsMode(Singleton):
+    type: MetricModeTypes
+    length: Optional[int] = None
+
+
+def match_metrics_mode(mode: str) -> MetricsMode:
+    sanitized_mode = mode.lower()

Review Comment:
   And trim?



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1285257956


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1039,335 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.values()).union({"INT96"})
+
+
+class StatsAggregator:
+    current_min: Any
+    current_max: Any
+    trunc_length: Optional[int]
+
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min = None
+        self.current_max = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        expected_physical_type = _PRIMITIVE_TO_PHYSICAL[iceberg_type]
+        if expected_physical_type != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {expected_physical_type}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:

Review Comment:
   I'm not sure `add` makes sense here. What about something like `update_min`?
   
   Also, in cases where this is used to accumulate a min and max from a list of values, should there be an `update` method that delegates to both `update_min` and `update_max`?



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1285261361


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1039,335 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.values()).union({"INT96"})
+
+
+class StatsAggregator:
+    current_min: Any
+    current_max: Any
+    trunc_length: Optional[int]
+
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min = None
+        self.current_max = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        expected_physical_type = _PRIMITIVE_TO_PHYSICAL[iceberg_type]
+        if expected_physical_type != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {expected_physical_type}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        self.current_min = val if self.current_min is None else min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        self.current_max = val if self.current_max is None else max(val, self.current_max)
+
+    def get_min(self) -> bytes:
+        return self.serialize(
+            self.current_min
+            if self.trunc_length is None
+            else TruncateTransform(width=self.trunc_length).transform(self.primitive_type)(self.current_min)
+        )
+
+    def get_max(self) -> Optional[bytes]:
+        if self.current_max is None:
+            return None
+
+        if self.primitive_type == StringType():
+            if type(self.current_max) != str:
+                raise ValueError("Expected the current_max to be a string")
+
+            s_result = self.current_max[: self.trunc_length]
+            if s_result != self.current_max:
+                chars = [*s_result]
+
+                for i in range(-1, -len(s_result) - 1, -1):
+                    try:
+                        to_inc = ord(chars[i])
+                        # will raise exception if the highest unicode code is reached
+                        _next = chr(to_inc + 1)
+                        chars[i] = _next
+                        return self.serialize("".join(chars))
+                    except ValueError:
+                        pass
+                return None  # didn't find a valid upper bound
+            return self.serialize(s_result)
+        elif self.primitive_type == BinaryType():
+            if type(self.current_max) != bytes:
+                raise ValueError("Expected the current_max to be bytes")
+            b_result = self.current_max[: self.trunc_length]
+            if b_result != self.current_max:
+                _bytes = [*b_result]
+                for i in range(-1, -len(b_result) - 1, -1):
+                    if _bytes[i] < 255:
+                        _bytes[i] += 1
+                        return b"".join([i.to_bytes(1, byteorder="little") for i in _bytes])
+                return None
+
+            return self.serialize(b_result)
+        else:
+            return self.serialize(self.current_max)[: self.trunc_length]
+
+
+DEFAULT_TRUNCATION_LENGTH = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+
+class MetricModeTypes(Enum):
+    TRUNCATE = "truncate"
+    NONE = "none"
+    COUNTS = "counts"
+    FULL = "full"
+
+
+DEFAULT_METRICS_MODE_KEY = "write.metadata.metrics.default"
+COLUMN_METRICS_MODE_KEY_PREFIX = "write.metadata.metrics.column"
+
+
+@dataclass(frozen=True)
+class MetricsMode(Singleton):
+    type: MetricModeTypes
+    length: Optional[int] = None
+
+
+def match_metrics_mode(mode: str) -> MetricsMode:
+    sanitized_mode = mode.lower()
+    if sanitized_mode.startswith("truncate"):
+        m = re.match(TRUNCATION_EXPR, mode, re.IGNORECASE)
+        if m:
+            length = int(m[1])
+            if length < 1:
+                raise ValueError("Truncation length must be larger than 0")
+            return MetricsMode(MetricModeTypes.TRUNCATE, int(m[1]))
+        else:
+            raise ValueError(f"Malformed truncate: {mode}")
+    elif sanitized_mode.startswith("none"):
+        return MetricsMode(MetricModeTypes.NONE)
+    elif sanitized_mode.startswith("counts"):
+        return MetricsMode(MetricModeTypes.COUNTS)
+    elif sanitized_mode.startswith("full"):
+        return MetricsMode(MetricModeTypes.FULL)
+    else:
+        raise ValueError(f"Unsupported metrics mode: {mode}")
+
+
+@dataclass(frozen=True)
+class StatisticsCollector:
+    field_id: int
+    iceberg_type: PrimitiveType
+    mode: MetricsMode
+    column_name: str
+
+
+class PyArrowStatisticsCollector(PreOrderSchemaVisitor[List[StatisticsCollector]]):
+    _field_id: int = 0
+    _schema: Schema
+    _properties: Dict[str, str]
+
+    def __init__(self, schema: Schema, properties: Dict[str, str]):
+        self._schema = schema
+        self._properties = properties
+
+    def schema(self, schema: Schema, struct_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        return struct_result()
+
+    def struct(
+        self, struct: StructType, field_results: List[Callable[[], List[StatisticsCollector]]]
+    ) -> List[StatisticsCollector]:
+        return list(chain(*[result() for result in field_results]))
+
+    def field(self, field: NestedField, field_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = field.field_id
+        result = field_result()
+        return result
+
+    def list(self, list_type: ListType, element_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = list_type.element_id
+        return element_result()
+
+    def map(
+        self,
+        map_type: MapType,
+        key_result: Callable[[], List[StatisticsCollector]],
+        value_result: Callable[[], List[StatisticsCollector]],
+    ) -> List[StatisticsCollector]:
+        self._field_id = map_type.key_id
+        k = key_result()
+        self._field_id = map_type.value_id
+        v = value_result()
+        return k + v
+
+    def primitive(self, primitive: PrimitiveType) -> List[StatisticsCollector]:
+        column_name = self._schema.find_column_name(self._field_id)
+        if column_name is None:
+            raise ValueError(f"Column for field {self._field_id} not found")
+
+        metrics_mode = MetricsMode(MetricModeTypes.TRUNCATE, DEFAULT_TRUNCATION_LENGTH)
+
+        default_mode = self._properties.get(DEFAULT_METRICS_MODE_KEY)
+        if default_mode:
+            metrics_mode = match_metrics_mode(default_mode)
+
+        col_mode = self._properties.get(f"{COLUMN_METRICS_MODE_KEY_PREFIX}.{column_name}")
+        if col_mode:
+            metrics_mode = match_metrics_mode(col_mode)
+
+        if (
+            not (isinstance(primitive, StringType) or isinstance(primitive, BinaryType))
+            and metrics_mode.type == MetricModeTypes.TRUNCATE
+        ):
+            metrics_mode = MetricsMode(MetricModeTypes.FULL)
+
+        return [StatisticsCollector(field_id=self._field_id, iceberg_type=primitive, mode=metrics_mode, column_name=column_name)]
+
+
+def fill_parquet_file_metadata(
+    df: DataFile,
+    parquet_metadata: pq.FileMetaData,
+    file_size: int,
+    table_metadata: TableMetadata,
+) -> None:
+    """
+    Computes and fills the following fields of the DataFile object.
+
+    - file_format
+    - record_count
+    - file_size_in_bytes
+    - column_sizes
+    - value_counts
+    - null_value_counts
+    - nan_value_counts
+    - lower_bounds
+    - upper_bounds
+    - split_offsets
+
+    Args:
+        df (DataFile): A DataFile object representing the Parquet file for which metadata is to be filled.
+        parquet_metadata (pyarrow.parquet.FileMetaData): A pyarrow metadata object.
+        file_size (int): The total compressed file size cannot be retrieved from the metadata and hence has to
+            be passed here. Depending on the kind of file system and pyarrow library call used, different
+            ways to obtain this value might be appropriate.
+        table_metadata (pyiceberg.table.metadata.TableMetadata): The Iceberg table metadata. It is required to
+            compute the mapping if column position to iceberg schema type id. It's also used to set the mode
+            for column metrics collection
+    """
+    schema = next(filter(lambda s: s.schema_id == table_metadata.current_schema_id, table_metadata.schemas))
+
+    stats_columns = pre_order_visit(schema, PyArrowStatisticsCollector(schema, table_metadata.properties))
+
+    if parquet_metadata.num_columns != len(stats_columns):
+        raise ValueError(
+            f"Number of columns in metadata ({len(stats_columns)}) is different from the number of columns in pyarrow table ({parquet_metadata.num_columns})"
+        )
+
+    column_sizes: Dict[int, int] = {}
+    value_counts: Dict[int, int] = {}
+    split_offsets: List[int] = []
+
+    null_value_counts: Dict[int, int] = {}
+    nan_value_counts: Dict[int, int] = {}
+
+    col_aggs = {}
+
+    for r in range(parquet_metadata.num_row_groups):
+        # References:
+        # https://github.com/apache/iceberg/blob/fc381a81a1fdb8f51a0637ca27cd30673bd7aad3/parquet/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java#L232
+        # https://github.com/apache/parquet-mr/blob/ac29db4611f86a07cc6877b416aa4b183e09b353/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/ColumnChunkMetaData.java#L184
+
+        row_group = parquet_metadata.row_group(r)
+
+        data_offset = row_group.column(0).data_page_offset
+        dictionary_offset = row_group.column(0).dictionary_page_offset
+
+        if row_group.column(0).has_dictionary_page and dictionary_offset < data_offset:
+            split_offsets.append(dictionary_offset)
+        else:
+            split_offsets.append(data_offset)
+
+        for pos, stats_col in enumerate(stats_columns):
+            field_id = stats_col.field_id
+
+            column = row_group.column(pos)
+
+            column_sizes[field_id] = column_sizes.get(field_id, 0) + column.total_compressed_size
+
+            if stats_col.mode == MetricsMode(MetricModeTypes.NONE):
+                continue
+
+            value_counts[field_id] = value_counts.get(field_id, 0) + column.num_values
+
+            if column.is_stats_set:
+                try:
+                    statistics = column.statistics
+
+                    null_value_counts[field_id] = null_value_counts.get(field_id, 0) + statistics.null_count
+
+                    if stats_col.mode == MetricsMode(MetricModeTypes.COUNTS):
+                        continue
+
+                    if field_id not in col_aggs:
+                        col_aggs[field_id] = StatsAggregator(
+                            stats_col.iceberg_type, statistics.physical_type, stats_col.mode.length
+                        )
+
+                    col_aggs[field_id].add_min(statistics.min)
+                    col_aggs[field_id].add_max(statistics.max)
+
+                except pyarrow.lib.ArrowNotImplementedError as e:
+                    logger.warning(e)
+            else:
+                logger.warning("PyArrow statistics missing for column %d when writing file", pos)
+
+    split_offsets.sort()
+
+    lower_bounds = {}
+    upper_bounds = {}
+
+    for k, agg in col_aggs.items():
+        lower_bounds[k] = agg.get_min()
+        _max = agg.get_max()
+        if _max is not None:
+            upper_bounds[k] = _max
+
+    df.file_format = FileFormat.PARQUET

Review Comment:
   @Fokko, are we okay with `DataFile` instances being mutable?



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1285261781


##########
python/tests/io/test_pyarrow.py:
##########
@@ -1345,3 +1374,655 @@ def test_pyarrow_wrap_fsspec(example_task: FileScanTask, table_schema_simple: Sc
 bar: [[1,2,3]]
 baz: [[true,false,null]]"""
     )
+
+
+def construct_test_table() -> Tuple[Any, Any, Union[TableMetadataV1, TableMetadataV2]]:

Review Comment:
   I think the tests for Parquet stats should go into a separate test file. There's too much in `test_pyarrow` as it is and this is a reasonably self-contained set of tests.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] maxdebayser commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "maxdebayser (via GitHub)" <gi...@apache.org>.
maxdebayser commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1288962269


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1039,335 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.values()).union({"INT96"})
+
+
+class StatsAggregator:
+    current_min: Any
+    current_max: Any
+    trunc_length: Optional[int]
+
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min = None
+        self.current_max = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        expected_physical_type = _PRIMITIVE_TO_PHYSICAL[iceberg_type]
+        if expected_physical_type != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {expected_physical_type}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        self.current_min = val if self.current_min is None else min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        self.current_max = val if self.current_max is None else max(val, self.current_max)
+
+    def get_min(self) -> bytes:
+        return self.serialize(
+            self.current_min
+            if self.trunc_length is None
+            else TruncateTransform(width=self.trunc_length).transform(self.primitive_type)(self.current_min)
+        )
+
+    def get_max(self) -> Optional[bytes]:
+        if self.current_max is None:
+            return None
+
+        if self.primitive_type == StringType():
+            if type(self.current_max) != str:
+                raise ValueError("Expected the current_max to be a string")
+
+            s_result = self.current_max[: self.trunc_length]
+            if s_result != self.current_max:
+                chars = [*s_result]
+
+                for i in range(-1, -len(s_result) - 1, -1):
+                    try:
+                        to_inc = ord(chars[i])
+                        # will raise exception if the highest unicode code is reached
+                        _next = chr(to_inc + 1)
+                        chars[i] = _next
+                        return self.serialize("".join(chars))
+                    except ValueError:
+                        pass
+                return None  # didn't find a valid upper bound
+            return self.serialize(s_result)
+        elif self.primitive_type == BinaryType():
+            if type(self.current_max) != bytes:
+                raise ValueError("Expected the current_max to be bytes")
+            b_result = self.current_max[: self.trunc_length]
+            if b_result != self.current_max:
+                _bytes = [*b_result]
+                for i in range(-1, -len(b_result) - 1, -1):
+                    if _bytes[i] < 255:
+                        _bytes[i] += 1
+                        return b"".join([i.to_bytes(1, byteorder="little") for i in _bytes])
+                return None
+
+            return self.serialize(b_result)
+        else:
+            return self.serialize(self.current_max)[: self.trunc_length]
+
+
+DEFAULT_TRUNCATION_LENGTH = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+
+class MetricModeTypes(Enum):
+    TRUNCATE = "truncate"
+    NONE = "none"
+    COUNTS = "counts"
+    FULL = "full"
+
+
+DEFAULT_METRICS_MODE_KEY = "write.metadata.metrics.default"
+COLUMN_METRICS_MODE_KEY_PREFIX = "write.metadata.metrics.column"
+
+
+@dataclass(frozen=True)
+class MetricsMode(Singleton):
+    type: MetricModeTypes
+    length: Optional[int] = None
+
+
+def match_metrics_mode(mode: str) -> MetricsMode:
+    sanitized_mode = mode.lower()
+    if sanitized_mode.startswith("truncate"):
+        m = re.match(TRUNCATION_EXPR, mode, re.IGNORECASE)
+        if m:
+            length = int(m[1])
+            if length < 1:
+                raise ValueError("Truncation length must be larger than 0")
+            return MetricsMode(MetricModeTypes.TRUNCATE, int(m[1]))
+        else:
+            raise ValueError(f"Malformed truncate: {mode}")
+    elif sanitized_mode.startswith("none"):
+        return MetricsMode(MetricModeTypes.NONE)
+    elif sanitized_mode.startswith("counts"):
+        return MetricsMode(MetricModeTypes.COUNTS)
+    elif sanitized_mode.startswith("full"):
+        return MetricsMode(MetricModeTypes.FULL)
+    else:
+        raise ValueError(f"Unsupported metrics mode: {mode}")
+
+
+@dataclass(frozen=True)
+class StatisticsCollector:
+    field_id: int
+    iceberg_type: PrimitiveType
+    mode: MetricsMode
+    column_name: str
+
+
+class PyArrowStatisticsCollector(PreOrderSchemaVisitor[List[StatisticsCollector]]):
+    _field_id: int = 0
+    _schema: Schema
+    _properties: Dict[str, str]
+
+    def __init__(self, schema: Schema, properties: Dict[str, str]):
+        self._schema = schema
+        self._properties = properties
+
+    def schema(self, schema: Schema, struct_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        return struct_result()
+
+    def struct(
+        self, struct: StructType, field_results: List[Callable[[], List[StatisticsCollector]]]
+    ) -> List[StatisticsCollector]:
+        return list(chain(*[result() for result in field_results]))
+
+    def field(self, field: NestedField, field_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = field.field_id
+        result = field_result()
+        return result
+
+    def list(self, list_type: ListType, element_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = list_type.element_id
+        return element_result()
+
+    def map(
+        self,
+        map_type: MapType,
+        key_result: Callable[[], List[StatisticsCollector]],
+        value_result: Callable[[], List[StatisticsCollector]],
+    ) -> List[StatisticsCollector]:
+        self._field_id = map_type.key_id
+        k = key_result()
+        self._field_id = map_type.value_id
+        v = value_result()
+        return k + v
+
+    def primitive(self, primitive: PrimitiveType) -> List[StatisticsCollector]:
+        column_name = self._schema.find_column_name(self._field_id)
+        if column_name is None:
+            raise ValueError(f"Column for field {self._field_id} not found")
+
+        metrics_mode = MetricsMode(MetricModeTypes.TRUNCATE, DEFAULT_TRUNCATION_LENGTH)
+
+        default_mode = self._properties.get(DEFAULT_METRICS_MODE_KEY)
+        if default_mode:
+            metrics_mode = match_metrics_mode(default_mode)
+
+        col_mode = self._properties.get(f"{COLUMN_METRICS_MODE_KEY_PREFIX}.{column_name}")
+        if col_mode:
+            metrics_mode = match_metrics_mode(col_mode)
+
+        if (
+            not (isinstance(primitive, StringType) or isinstance(primitive, BinaryType))
+            and metrics_mode.type == MetricModeTypes.TRUNCATE
+        ):
+            metrics_mode = MetricsMode(MetricModeTypes.FULL)
+
+        return [StatisticsCollector(field_id=self._field_id, iceberg_type=primitive, mode=metrics_mode, column_name=column_name)]
+
+
+def fill_parquet_file_metadata(
+    df: DataFile,
+    parquet_metadata: pq.FileMetaData,
+    file_size: int,
+    table_metadata: TableMetadata,
+) -> None:
+    """
+    Computes and fills the following fields of the DataFile object.
+
+    - file_format
+    - record_count
+    - file_size_in_bytes
+    - column_sizes
+    - value_counts
+    - null_value_counts
+    - nan_value_counts
+    - lower_bounds
+    - upper_bounds
+    - split_offsets
+
+    Args:
+        df (DataFile): A DataFile object representing the Parquet file for which metadata is to be filled.
+        parquet_metadata (pyarrow.parquet.FileMetaData): A pyarrow metadata object.
+        file_size (int): The total compressed file size cannot be retrieved from the metadata and hence has to
+            be passed here. Depending on the kind of file system and pyarrow library call used, different
+            ways to obtain this value might be appropriate.
+        table_metadata (pyiceberg.table.metadata.TableMetadata): The Iceberg table metadata. It is required to
+            compute the mapping if column position to iceberg schema type id. It's also used to set the mode
+            for column metrics collection
+    """
+    schema = next(filter(lambda s: s.schema_id == table_metadata.current_schema_id, table_metadata.schemas))

Review Comment:
   I agree, but I don't think it's in the scope of this PR.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on pull request #7831: Python: Compute parquet stats

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#issuecomment-1631183637

   For some reason my big comment was collapsed:
   
   Since Arrow is column-oriented, I'm sure that it will follow the order of the write schema:
   
   ![image](https://github.com/apache/iceberg/assets/1134248/5dd5016e-7761-4cdc-a169-d3c4744c581b)
   
   The reason I try to avoid using too many internal details from PyArrow is that we support PyArrow `[9.0.0, 12.0.1]` currently. There is no guarantee that all these internals stay the same, therefore I think we should do as much as possible within our own control (also regarding the internal Parquet types, or how the column names are structured).
   
   We already have the write schema, so we can easily filter out the primitive types:
   
   ```python
   class PyArrowStatisticsCollector(PreOrderSchemaVisitor[List[StatisticsCollector]]):
       _field_id = 0
       _schema: Schema
       _properties: Properties
   
       def __init__(self, schema: Schema, properties: Dict[str, str]):
           self._schema = schema
           self._properties = properties
   
       def schema(self, schema: Schema, struct_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
           return struct_result()
   
       def struct(self, struct: StructType, field_results: List[Callable[[], List[StatisticsCollector]]]) -> List[StatisticsCollector]:
           return list(chain(*[result() for result in field_results]))
   
       def field(self, field: NestedField, field_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
           self._field_id = field.field_id
           result = field_result()
           return result
   
       def list(self, list_type: ListType, element_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
           self._field_id = list_type.element_id
           return element_result()
   
       def map(self, map_type: MapType, key_result: Callable[[], List[StatisticsCollector]], value_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
           self._field_id = map_type.key_id
           k = key_result()
           self._field_id = map_type.value_id
           v = value_result()
           return k + v
   
       def primitive(self, primitive: PrimitiveType) -> List[StatisticsCollector]:
           return [StatisticsCollector(
               field_id=self._field_id,
               iceberg_type=primitive,
               mode=MetricsMode.TRUNC
               # schema
               # self._properties.get(f"write.metadata.metrics.column.{schema.find_column_name(self._field_id)}")
           )]
   ```
   
   This way we get a nice list of columns that we need to collect statistics for. We have:
   
   ```python
   @dataclass(frozen=True)
   class StatisticsCollector:
       field_id: int
       iceberg_type: PrimitiveType
       mode: MetricsMode
   ```
   
   Where we can use the `field_id` to properly populate the maps, and the `iceberg_type` to feed into `to_bytes` to do the conversion (so we don't have to have yet another one for PyArrow).
   
   I did a quick test, and it seems to work:
   ```python
   def test_complex_schema(table_schema_nested: Schema):
       tbl = pa.Table.from_pydict({
           "foo": ["a", "b"],
           "bar": [1, 2],
           "baz": [False, True],
           "qux": [["a", "b"], ["c", "d"]],
           "quux": [[("a", (("aa", 1), ("ab", 2)))], [("b", (("ba", 3), ("bb", 4)))]],
           "location": [[(52.377956, 4.897070), (4.897070, -122.431297)],
                        [(43.618881, -116.215019), (41.881832, -87.623177)]],
           "person": [("Fokko", 33), ("Max", 42)]  # Possible data quality issue
       },
           schema=schema_to_pyarrow(table_schema_nested)
       )
       stats_columns = pre_order_visit(table_schema_nested, PyArrowStatisticsCollector(table_schema_nested, {}))
   
       visited_paths = []
   
       def file_visitor(written_file: Any) -> None:
           visited_paths.append(written_file)
   
       with TemporaryDirectory() as tmpdir:
           pq.write_to_dataset(tbl, tmpdir, file_visitor=file_visitor)
   
       assert visited_paths[0].metadata.num_columns == len(stats_columns)
   ```


-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on pull request #7831: Python: Compute parquet stats

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#issuecomment-1727496620

   @maxdebayser Thanks for working on this, and for your patience because it took quite a while, and we also went a bit back and forth. Let's merge this in. As a next step, we can revisit https://github.com/apache/iceberg/pull/8012 and add a lot of integration tests to make sure that we don't have any correctness issues. 
   
   I think there are still some open-ends on the statistics, but they are not limited to Python. Please refer to https://github.com/apache/iceberg/issues/8598


-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1259416267


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1013,3 +1027,359 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+BOUND_TRUNCATED_LENGHT = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+logger = logging.getLogger(__name__)
+
+# Serialization rules: https://iceberg.apache.org/spec/#binary-single-value-serialization
+#
+# Type      Binary serialization
+# boolean   0x00 for false, non-zero byte for true
+# int       Stored as 4-byte little-endian
+# long      Stored as 8-byte little-endian
+# float     Stored as 4-byte little-endian
+# double    Stored as 8-byte little-endian
+# date      Stores days from the 1970-01-01 in an 4-byte little-endian int
+# time      Stores microseconds from midnight in an 8-byte little-endian long
+# timestamp without zone	Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long
+# timestamp with zone	Stores microseconds from 1970-01-01 00:00:00.000000 UTC in an 8-byte little-endian long
+# string    UTF-8 bytes (without length)
+# uuid      16-byte big-endian value, see example in Appendix B
+# fixed(L)  Binary value
+# binary    Binary value (without length)
+#
+
+
+def bool_to_avro(value: bool) -> bytes:
+    return STRUCT_BOOL.pack(value)
+
+
+def int32_to_avro(value: int) -> bytes:
+    return STRUCT_INT32.pack(value)
+
+
+def int64_to_avro(value: int) -> bytes:
+    return STRUCT_INT64.pack(value)
+
+
+def float_to_avro(value: float) -> bytes:
+    return STRUCT_FLOAT.pack(value)
+
+
+def double_to_avro(value: float) -> bytes:
+    return STRUCT_DOUBLE.pack(value)
+
+
+def bytes_to_avro(value: Union[bytes, str]) -> bytes:
+    if type(value) == str:
+        return value.encode()
+    else:
+        assert isinstance(value, bytes)  # appeases mypy
+        return value
+
+
+class StatsAggregator:
+    def __init__(self, type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.serialize: Any = None
+        self.trunc_lenght = trunc_length
+
+        if type_string == "BOOLEAN":
+            self.serialize = bool_to_avro
+        elif type_string == "INT32":
+            self.serialize = int32_to_avro
+        elif type_string == "INT64":
+            self.serialize = int64_to_avro
+        elif type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+        elif type_string == "FLOAT":
+            self.serialize = float_to_avro
+        elif type_string == "DOUBLE":
+            self.serialize = double_to_avro
+        elif type_string == "BYTE_ARRAY":
+            self.serialize = bytes_to_avro
+        elif type_string == "FIXED_LEN_BYTE_ARRAY":
+            self.serialize = bytes_to_avro
+        else:
+            raise AssertionError(f"Unknown physical type {type_string}")
+
+    def add_min(self, val: bytes) -> None:
+        if self.current_min is None:
+            self.current_min = val
+        else:
+            self.current_min = min(val, self.current_min)
+
+    def add_max(self, val: bytes) -> None:
+        if self.current_max is None:
+            self.current_max = val
+        else:
+            self.current_max = max(self.current_max, val)
+
+    def get_min(self) -> bytes:
+        return self.serialize(self.current_min)[: self.trunc_lenght]
+
+    def get_max(self) -> bytes:
+        return self.serialize(self.current_max)[: self.trunc_lenght]
+
+
+class MetricsMode(Enum):
+    NONE = 0
+    COUNTS = 1
+    TRUNC = 2

Review Comment:
   ```suggestion
       TRUNCATE = 2
   ```



##########
python/tests/io/test_pyarrow.py:
##########
@@ -1330,3 +1341,485 @@ def test_pyarrow_wrap_fsspec(example_task: FileScanTask, table_schema_simple: Sc
 bar: [[1,2,3]]
 baz: [[true,false,null]]"""
     )
+
+
+def construct_test_table() -> pa.Buffer:
+    schema = pa.schema(
+        [
+            pa.field("strings", pa.string()),
+            pa.field("floats", pa.float64()),
+            pa.field("list", pa.list_(pa.int64())),
+            pa.field("maps", pa.map_(pa.int64(), pa.int64())),
+        ]
+    )
+
+    _strings = ["zzzzzzzzzzzzzzzzzzzz", "rrrrrrrrrrrrrrrrrrrr", None, "aaaaaaaaaaaaaaaaaaaa"]
+
+    _floats = [3.14, math.nan, 1.69, 100]
+
+    _list = [[1, 2, 3], [4, 5, 6], None, [7, 8, 9]]
+
+    _maps: List[Optional[Dict[int, int]]] = [
+        {1: 2, 3: 4},
+        None,
+        {5: 6},
+        {},
+    ]
+
+    table = pa.Table.from_pydict(
+        {
+            "strings": _strings,
+            "floats": _floats,
+            "list": _list,
+            "maps": _maps,
+        },
+        schema=schema,
+    )
+    f = pa.BufferOutputStream()
+
+    metadata_collector: List[Any] = []
+    writer = pq.ParquetWriter(f, table.schema, metadata_collector=metadata_collector)
+
+    writer.write_table(table)
+    writer.close()
+
+    print(writer.writer)
+    print(writer.writer.metadata)
+
+    mapping = {"strings": 1, "floats": 2, "list.list.item": 3, "maps.key_value.key": 4, "maps.key_value.value": 5}
+
+    return f.getvalue(), metadata_collector[0], mapping
+
+
+def test_record_count() -> None:
+    (file_bytes, metadata, mapping) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, mapping, len(file_bytes))
+
+    assert datafile.record_count == 4
+
+
+def test_file_size() -> None:
+    (file_bytes, metadata, mapping) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, mapping, len(file_bytes))
+
+    assert datafile.file_size_in_bytes == len(file_bytes)
+
+
+def test_value_counts() -> None:
+    (file_bytes, metadata, mapping) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, mapping, len(file_bytes))
+
+    assert len(datafile.value_counts) == 5
+    assert datafile.value_counts[1] == 4
+    assert datafile.value_counts[2] == 4
+    assert datafile.value_counts[3] == 10  # 3 lists with 3 items and a None value
+    assert datafile.value_counts[4] == 5
+    assert datafile.value_counts[5] == 5
+
+
+def test_column_sizes() -> None:
+    (file_bytes, metadata, mapping) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, mapping, len(file_bytes))
+
+    assert len(datafile.column_sizes) == 5
+    # these values are an artifact of how the write_table encodes the columns
+    assert datafile.column_sizes[1] == 116
+    assert datafile.column_sizes[2] == 119
+    assert datafile.column_sizes[3] == 151
+    assert datafile.column_sizes[4] == 117
+    assert datafile.column_sizes[5] == 117
+
+
+def test_null_and_nan_counts() -> None:
+    (file_bytes, metadata, mapping) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, mapping, len(file_bytes))
+
+    assert len(datafile.null_value_counts) == 5
+    assert datafile.null_value_counts[1] == 1
+    assert datafile.null_value_counts[2] == 0
+    assert datafile.null_value_counts[3] == 1
+    assert datafile.null_value_counts[4] == 2
+    assert datafile.null_value_counts[5] == 2
+
+    # #arrow does not include this in the statistics
+    # assert len(datafile.nan_value_counts)  == 3
+    # assert datafile.nan_value_counts[1]    == 0
+    # assert datafile.nan_value_counts[2]    == 1
+    # assert datafile.nan_value_counts[3]    == 0
+
+
+def test_bounds() -> None:
+    (file_bytes, metadata, mapping) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, mapping, len(file_bytes))
+
+    assert len(datafile.lower_bounds) == 2
+    assert datafile.lower_bounds[1].decode() == "aaaaaaaaaaaaaaaaaaaa"[:BOUND_TRUNCATED_LENGHT]
+    assert datafile.lower_bounds[2] == STRUCT_DOUBLE.pack(1.69)
+
+    assert len(datafile.upper_bounds) == 2
+    assert datafile.upper_bounds[1].decode() == "zzzzzzzzzzzzzzzzzzzz"[:BOUND_TRUNCATED_LENGHT]
+    assert datafile.upper_bounds[2] == STRUCT_DOUBLE.pack(100)
+
+
+def test_metrics_mode_none(example_table_metadata_v2: Dict[str, Any]) -> None:
+    (file_bytes, metadata, mapping) = construct_test_table()
+
+    datafile = DataFile()
+    table_metadata = TableMetadataUtil.parse_obj(example_table_metadata_v2)
+    table_metadata.properties["write.metadata.metrics.default"] = "none"
+    fill_parquet_file_metadata(
+        datafile,
+        metadata,
+        mapping,
+        len(file_bytes),
+        table_metadata,
+    )
+
+    assert len(datafile.value_counts) == 0
+    assert len(datafile.null_value_counts) == 0
+    assert len(datafile.nan_value_counts) == 0
+    assert len(datafile.lower_bounds) == 0
+    assert len(datafile.upper_bounds) == 0
+
+
+def test_metrics_mode_counts(example_table_metadata_v2: Dict[str, Any]) -> None:
+    (file_bytes, metadata, mapping) = construct_test_table()
+
+    datafile = DataFile()
+    table_metadata = TableMetadataUtil.parse_obj(example_table_metadata_v2)
+    table_metadata.properties["write.metadata.metrics.default"] = "counts"
+    fill_parquet_file_metadata(
+        datafile,
+        metadata,
+        mapping,
+        len(file_bytes),
+        table_metadata,
+    )
+
+    assert len(datafile.value_counts) == 5
+    assert len(datafile.null_value_counts) == 5
+    assert len(datafile.nan_value_counts) == 0
+    assert len(datafile.lower_bounds) == 0
+    assert len(datafile.upper_bounds) == 0
+
+
+def test_metrics_mode_full(example_table_metadata_v2: Dict[str, Any]) -> None:
+    (file_bytes, metadata, mapping) = construct_test_table()
+
+    datafile = DataFile()
+    table_metadata = TableMetadataUtil.parse_obj(example_table_metadata_v2)
+    table_metadata.properties["write.metadata.metrics.default"] = "full"
+    fill_parquet_file_metadata(
+        datafile,
+        metadata,
+        mapping,
+        len(file_bytes),
+        table_metadata,
+    )
+
+    assert len(datafile.value_counts) == 5
+    assert len(datafile.null_value_counts) == 5
+    assert len(datafile.nan_value_counts) == 0
+
+    assert len(datafile.lower_bounds) == 2
+    assert datafile.lower_bounds[1].decode() == "aaaaaaaaaaaaaaaaaaaa"
+    assert datafile.lower_bounds[2] == STRUCT_DOUBLE.pack(1.69)
+
+    assert len(datafile.upper_bounds) == 2
+    assert datafile.upper_bounds[1].decode() == "zzzzzzzzzzzzzzzzzzzz"
+    assert datafile.upper_bounds[2] == STRUCT_DOUBLE.pack(100)
+
+
+def test_metrics_mode_non_default_trunc(example_table_metadata_v2: Dict[str, Any]) -> None:
+    (file_bytes, metadata, mapping) = construct_test_table()
+
+    datafile = DataFile()
+    table_metadata = TableMetadataUtil.parse_obj(example_table_metadata_v2)
+    table_metadata.properties["write.metadata.metrics.default"] = "truncate(2)"
+    fill_parquet_file_metadata(
+        datafile,
+        metadata,
+        mapping,
+        len(file_bytes),
+        table_metadata,
+    )
+
+    assert len(datafile.value_counts) == 5
+    assert len(datafile.null_value_counts) == 5
+    assert len(datafile.nan_value_counts) == 0
+
+    assert len(datafile.lower_bounds) == 2
+    assert datafile.lower_bounds[1].decode() == "aa"
+    assert datafile.lower_bounds[2] == STRUCT_DOUBLE.pack(1.69)[:2]
+
+    assert len(datafile.upper_bounds) == 2
+    assert datafile.upper_bounds[1].decode() == "zz"
+    assert datafile.upper_bounds[2] == STRUCT_DOUBLE.pack(100)[:2]
+
+
+def test_column_metrics_mode(example_table_metadata_v2: Dict[str, Any]) -> None:
+    (file_bytes, metadata, mapping) = construct_test_table()
+
+    datafile = DataFile()
+    table_metadata = TableMetadataUtil.parse_obj(example_table_metadata_v2)
+    table_metadata.properties["write.metadata.metrics.default"] = "truncate(2)"
+    table_metadata.properties["write.metadata.metrics.column.strings"] = "none"
+    fill_parquet_file_metadata(
+        datafile,
+        metadata,
+        mapping,
+        len(file_bytes),
+        table_metadata,
+    )
+
+    assert len(datafile.value_counts) == 4
+    assert len(datafile.null_value_counts) == 4
+    assert len(datafile.nan_value_counts) == 0
+
+    assert len(datafile.lower_bounds) == 1
+    assert datafile.lower_bounds[2] == STRUCT_DOUBLE.pack(1.69)[:2]
+
+    assert len(datafile.upper_bounds) == 1
+    assert datafile.upper_bounds[2] == STRUCT_DOUBLE.pack(100)[:2]
+
+
+def test_offsets() -> None:
+    (file_bytes, metadata, mapping) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, mapping, len(file_bytes))
+
+    assert datafile.split_offsets is not None
+    assert len(datafile.split_offsets) == 1
+    assert datafile.split_offsets[0] == 4
+
+
+def test_dataset() -> pa.Buffer:
+    schema = pa.schema([pa.field("ints", pa.int64()), pa.field("even", pa.bool_())])
+
+    _ints = [0, 2, 4, 8, 1, 3, 5, 7]
+    parity = [True, True, True, True, False, False, False, False]
+
+    table = pa.Table.from_pydict({"ints": _ints, "even": parity}, schema=schema)
+
+    visited_paths = []
+
+    def file_visitor(written_file: Any) -> None:
+        visited_paths.append(written_file)
+
+    with TemporaryDirectory() as tmpdir:
+        pq.write_to_dataset(table, tmpdir, partition_cols=["even"], file_visitor=file_visitor)
+
+    even = None
+    odd = None
+
+    assert len(visited_paths) == 2
+
+    for written_file in visited_paths:
+        df = DataFile()
+
+        fill_parquet_file_metadata(df, written_file.metadata, {"ints": 1, "even": 2}, written_file.size)
+
+        if "even=true" in written_file.path:
+            even = df
+
+        if "even=false" in written_file.path:
+            odd = df
+
+    assert even is not None
+    assert odd is not None
+
+    assert len(even.value_counts) == 1
+    assert even.value_counts[1] == 4
+    assert len(even.lower_bounds) == 1
+    assert even.lower_bounds[1] == STRUCT_INT64.pack(0)
+    assert len(even.upper_bounds) == 1
+    assert even.upper_bounds[1] == STRUCT_INT64.pack(8)
+
+    assert len(odd.value_counts) == 1
+    assert odd.value_counts[1] == 4
+    assert len(odd.lower_bounds) == 1
+    assert odd.lower_bounds[1] == STRUCT_INT64.pack(1)
+    assert len(odd.upper_bounds) == 1
+    assert odd.upper_bounds[1] == STRUCT_INT64.pack(7)
+
+
+def test_schema_mapping() -> None:

Review Comment:
   You can re-use an existing fixture here.
   ```suggestion
   def test_schema_mapping(table_schema_nested: Schema) -> None:
   ```



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1013,3 +1027,359 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+BOUND_TRUNCATED_LENGHT = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+logger = logging.getLogger(__name__)
+
+# Serialization rules: https://iceberg.apache.org/spec/#binary-single-value-serialization
+#
+# Type      Binary serialization
+# boolean   0x00 for false, non-zero byte for true
+# int       Stored as 4-byte little-endian
+# long      Stored as 8-byte little-endian
+# float     Stored as 4-byte little-endian
+# double    Stored as 8-byte little-endian
+# date      Stores days from the 1970-01-01 in an 4-byte little-endian int
+# time      Stores microseconds from midnight in an 8-byte little-endian long
+# timestamp without zone	Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long
+# timestamp with zone	Stores microseconds from 1970-01-01 00:00:00.000000 UTC in an 8-byte little-endian long
+# string    UTF-8 bytes (without length)
+# uuid      16-byte big-endian value, see example in Appendix B
+# fixed(L)  Binary value
+# binary    Binary value (without length)
+#
+
+
+def bool_to_avro(value: bool) -> bytes:
+    return STRUCT_BOOL.pack(value)
+
+
+def int32_to_avro(value: int) -> bytes:
+    return STRUCT_INT32.pack(value)
+
+
+def int64_to_avro(value: int) -> bytes:
+    return STRUCT_INT64.pack(value)
+
+
+def float_to_avro(value: float) -> bytes:
+    return STRUCT_FLOAT.pack(value)
+
+
+def double_to_avro(value: float) -> bytes:
+    return STRUCT_DOUBLE.pack(value)
+
+
+def bytes_to_avro(value: Union[bytes, str]) -> bytes:
+    if type(value) == str:
+        return value.encode()
+    else:
+        assert isinstance(value, bytes)  # appeases mypy
+        return value
+
+
+class StatsAggregator:
+    def __init__(self, type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.serialize: Any = None
+        self.trunc_lenght = trunc_length
+
+        if type_string == "BOOLEAN":
+            self.serialize = bool_to_avro
+        elif type_string == "INT32":
+            self.serialize = int32_to_avro
+        elif type_string == "INT64":
+            self.serialize = int64_to_avro
+        elif type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+        elif type_string == "FLOAT":
+            self.serialize = float_to_avro
+        elif type_string == "DOUBLE":
+            self.serialize = double_to_avro
+        elif type_string == "BYTE_ARRAY":
+            self.serialize = bytes_to_avro
+        elif type_string == "FIXED_LEN_BYTE_ARRAY":
+            self.serialize = bytes_to_avro
+        else:
+            raise AssertionError(f"Unknown physical type {type_string}")
+
+    def add_min(self, val: bytes) -> None:
+        if self.current_min is None:
+            self.current_min = val
+        else:
+            self.current_min = min(val, self.current_min)
+
+    def add_max(self, val: bytes) -> None:
+        if self.current_max is None:
+            self.current_max = val
+        else:
+            self.current_max = max(self.current_max, val)
+
+    def get_min(self) -> bytes:
+        return self.serialize(self.current_min)[: self.trunc_lenght]
+
+    def get_max(self) -> bytes:
+        return self.serialize(self.current_max)[: self.trunc_lenght]
+
+
+class MetricsMode(Enum):
+    NONE = 0
+    COUNTS = 1
+    TRUNC = 2
+    FULL = 3
+
+
+def match_metrics_mode(mode: str) -> Tuple[MetricsMode, Optional[int]]:
+    m = re.match(TRUNCATION_EXPR, mode)
+
+    if m:
+        return MetricsMode.TRUNC, int(m[1])
+    elif mode == "none":
+        return MetricsMode.NONE, None
+    elif mode == "counts":
+        return MetricsMode.COUNTS, None
+    elif mode == "full":
+        return MetricsMode.FULL, None
+    else:
+        raise AssertionError(f"Unsupported metrics mode {mode}")
+
+
+def fill_parquet_file_metadata(
+    df: DataFile,
+    metadata: pq.FileMetaData,
+    col_path_2_iceberg_id: Dict[str, int],
+    file_size: int,
+    table_metadata: Optional[TableMetadata] = None,
+) -> None:
+    """
+    Computes and fills the following fields of the DataFile object.
+
+    - file_format
+    - record_count
+    - file_size_in_bytes
+    - column_sizes
+    - value_counts
+    - null_value_counts
+    - nan_value_counts
+    - lower_bounds
+    - upper_bounds
+    - split_offsets
+
+    Args:
+        df (DataFile): A DataFile object representing the Parquet file for which metadata is to be filled.
+        metadata (pyarrow.parquet.FileMetaData): A pyarrow metadata object.
+        col_path_2_iceberg_id: A mapping of column paths as in the `path_in_schema` attribute of the colum
+            metadata to iceberg schema IDs. For scalar columns this will be the column name. For complex types
+            it could be something like `my_map.key_value.value`
+        file_size (int): The total compressed file size cannot be retrieved from the metadata and hence has to
+            be passed here. Depending on the kind of file system and pyarrow library call used, different
+            ways to obtain this value might be appropriate.
+    """
+    col_index_2_id = {}
+
+    col_names = {p.split(".")[0] for p in col_path_2_iceberg_id.keys()}
+
+    metrics_modes = {n: MetricsMode.TRUNC for n in col_names}
+    trunc_lengths: Dict[str, Optional[int]] = {n: BOUND_TRUNCATED_LENGHT for n in col_names}
+
+    if table_metadata:
+        default_mode = table_metadata.properties.get("write.metadata.metrics.default")

Review Comment:
   Can we move this one into a constant?



##########
python/pyiceberg/utils/file_stats.py:
##########
@@ -0,0 +1,333 @@
+#  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.
+
+import struct
+from typing import (
+    Any,
+    Dict,
+    List,
+    Union,
+)
+
+import pyarrow.lib
+import pyarrow.parquet as pq
+
+from pyiceberg.manifest import DataFile, FileFormat
+from pyiceberg.schema import Schema, SchemaVisitor, visit
+from pyiceberg.types import (
+    IcebergType,
+    ListType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StructType,
+)
+
+BOUND_TRUNCATED_LENGHT = 16
+
+# Serialization rules: https://iceberg.apache.org/spec/#binary-single-value-serialization
+#
+# Type      Binary serialization
+# boolean   0x00 for false, non-zero byte for true
+# int       Stored as 4-byte little-endian
+# long      Stored as 8-byte little-endian
+# float     Stored as 4-byte little-endian
+# double    Stored as 8-byte little-endian
+# date      Stores days from the 1970-01-01 in an 4-byte little-endian int
+# time      Stores microseconds from midnight in an 8-byte little-endian long
+# timestamp without zone	Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long
+# timestamp with zone	Stores microseconds from 1970-01-01 00:00:00.000000 UTC in an 8-byte little-endian long
+# string    UTF-8 bytes (without length)
+# uuid      16-byte big-endian value, see example in Appendix B
+# fixed(L)  Binary value
+# binary    Binary value (without length)
+#
+
+
+def bool_to_avro(value: bool) -> bytes:
+    return struct.pack("?", value)
+
+
+def int32_to_avro(value: int) -> bytes:
+    return struct.pack("<i", value)
+
+
+def int64_to_avro(value: int) -> bytes:
+    return struct.pack("<q", value)
+
+
+def float_to_avro(value: float) -> bytes:
+    return struct.pack("<f", value)
+
+
+def double_to_avro(value: float) -> bytes:
+    return struct.pack("<d", value)
+
+
+def bytes_to_avro(value: Union[bytes, str]) -> bytes:
+    if type(value) == str:
+        return value.encode()
+    else:
+        assert isinstance(value, bytes)  # appeases mypy
+        return value
+
+
+class StatsAggregator:
+    def __init__(self, type_string: str):
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.serialize: Any = None
+
+        if type_string == "BOOLEAN":
+            self.serialize = bool_to_avro
+        elif type_string == "INT32":
+            self.serialize = int32_to_avro
+        elif type_string == "INT64":
+            self.serialize = int64_to_avro
+        elif type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+        elif type_string == "FLOAT":
+            self.serialize = float_to_avro
+        elif type_string == "DOUBLE":
+            self.serialize = double_to_avro
+        elif type_string == "BYTE_ARRAY":
+            self.serialize = bytes_to_avro
+        elif type_string == "FIXED_LEN_BYTE_ARRAY":
+            self.serialize = bytes_to_avro
+        else:
+            raise AssertionError(f"Unknown physical type {type_string}")
+
+    def add_min(self, val: bytes) -> None:
+        if not self.current_min:
+            self.current_min = val
+        elif val < self.current_min:
+            self.current_min = val
+
+    def add_max(self, val: bytes) -> None:
+        if not self.current_max:
+            self.current_max = val
+        elif self.current_max < val:
+            self.current_max = val
+
+    def get_min(self) -> bytes:
+        return self.serialize(self.current_min)[:BOUND_TRUNCATED_LENGHT]
+
+    def get_max(self) -> bytes:
+        return self.serialize(self.current_max)[:BOUND_TRUNCATED_LENGHT]
+
+
+def fill_parquet_file_metadata(
+    df: DataFile, metadata: pq.FileMetaData, col_path_2_iceberg_id: Dict[str, int], file_size: int
+) -> None:
+    """
+    Computes and fills the following fields of the DataFile object.
+
+    - file_format
+    - record_count
+    - file_size_in_bytes
+    - column_sizes
+    - value_counts
+    - null_value_counts
+    - nan_value_counts
+    - lower_bounds
+    - upper_bounds
+    - split_offsets
+
+    Args:
+        df (DataFile): A DataFile object representing the Parquet file for which metadata is to be filled.
+        metadata (pyarrow.parquet.FileMetaData): A pyarrow metadata object.
+        col_path_2_iceberg_id: A mapping of column paths as in the `path_in_schema` attribute of the colum
+            metadata to iceberg schema IDs. For scalar columns this will be the column name. For complex types
+            it could be something like `my_map.key_value.value`
+        file_size (int): The total compressed file size cannot be retrieved from the metadata and hence has to
+            be passed here. Depending on the kind of file system and pyarrow library call used, different
+            ways to obtain this value might be appropriate.
+    """
+    col_index_2_id = {}
+
+    col_names = set(metadata.schema.names)
+
+    first_group = metadata.row_group(0)
+
+    for c in range(metadata.num_columns):
+        column = first_group.column(c)
+        col_path = column.path_in_schema
+
+        if col_path in col_path_2_iceberg_id:
+            col_index_2_id[c] = col_path_2_iceberg_id[col_path]
+        else:
+            raise AssertionError(f"Column path {col_path} couldn't be mapped to an iceberg ID")
+
+    column_sizes: Dict[int, int] = {}
+    value_counts: Dict[int, int] = {}
+    split_offsets: List[int] = []
+
+    null_value_counts: Dict[int, int] = {}
+    nan_value_counts: Dict[int, int] = {}
+
+    col_aggs = {}
+
+    for r in range(metadata.num_row_groups):
+        # References:
+        # https://github.com/apache/iceberg/blob/fc381a81a1fdb8f51a0637ca27cd30673bd7aad3/parquet/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java#L232
+        # https://github.com/apache/parquet-mr/blob/ac29db4611f86a07cc6877b416aa4b183e09b353/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/ColumnChunkMetaData.java#L184
+
+        row_group = metadata.row_group(r)
+
+        data_offset = row_group.column(0).data_page_offset
+        dictionary_offset = row_group.column(0).dictionary_page_offset
+
+        if row_group.column(0).has_dictionary_page and dictionary_offset < data_offset:
+            split_offsets.append(dictionary_offset)
+        else:
+            split_offsets.append(data_offset)
+
+        for c in range(metadata.num_columns):
+            col_id = col_index_2_id[c]
+
+            column = row_group.column(c)
+
+            column_sizes[col_id] = column_sizes.get(col_id, 0) + column.total_compressed_size
+            value_counts[col_id] = value_counts.get(col_id, 0) + column.num_values
+
+            if column.is_stats_set:
+                try:
+                    statistics = column.statistics
+
+                    null_value_counts[col_id] = null_value_counts.get(col_id, 0) + statistics.null_count
+
+                    if column.path_in_schema in col_names:
+                        # Iceberg seems to only have statistics for scalar columns
+
+                        if col_id not in col_aggs:
+                            col_aggs[col_id] = StatsAggregator(statistics.physical_type)
+
+                        col_aggs[col_id].add_min(statistics.min)

Review Comment:
   Fair point, thanks for the context



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1013,3 +1027,359 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+BOUND_TRUNCATED_LENGHT = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+logger = logging.getLogger(__name__)
+
+# Serialization rules: https://iceberg.apache.org/spec/#binary-single-value-serialization
+#
+# Type      Binary serialization
+# boolean   0x00 for false, non-zero byte for true
+# int       Stored as 4-byte little-endian
+# long      Stored as 8-byte little-endian
+# float     Stored as 4-byte little-endian
+# double    Stored as 8-byte little-endian
+# date      Stores days from the 1970-01-01 in an 4-byte little-endian int
+# time      Stores microseconds from midnight in an 8-byte little-endian long
+# timestamp without zone	Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long
+# timestamp with zone	Stores microseconds from 1970-01-01 00:00:00.000000 UTC in an 8-byte little-endian long
+# string    UTF-8 bytes (without length)
+# uuid      16-byte big-endian value, see example in Appendix B
+# fixed(L)  Binary value
+# binary    Binary value (without length)
+#
+
+
+def bool_to_avro(value: bool) -> bytes:

Review Comment:
   When we use the Iceberg types, then we can re-use `to_bytes`:
   
   https://github.com/apache/iceberg/blob/62cb7b5c50451085d6e59f0594292bf883af4220/python/pyiceberg/conversions.py#L154-L166



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1013,3 +1027,359 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+BOUND_TRUNCATED_LENGHT = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+logger = logging.getLogger(__name__)
+
+# Serialization rules: https://iceberg.apache.org/spec/#binary-single-value-serialization
+#
+# Type      Binary serialization
+# boolean   0x00 for false, non-zero byte for true
+# int       Stored as 4-byte little-endian
+# long      Stored as 8-byte little-endian
+# float     Stored as 4-byte little-endian
+# double    Stored as 8-byte little-endian
+# date      Stores days from the 1970-01-01 in an 4-byte little-endian int
+# time      Stores microseconds from midnight in an 8-byte little-endian long
+# timestamp without zone	Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long
+# timestamp with zone	Stores microseconds from 1970-01-01 00:00:00.000000 UTC in an 8-byte little-endian long
+# string    UTF-8 bytes (without length)
+# uuid      16-byte big-endian value, see example in Appendix B
+# fixed(L)  Binary value
+# binary    Binary value (without length)
+#
+
+
+def bool_to_avro(value: bool) -> bytes:
+    return STRUCT_BOOL.pack(value)
+
+
+def int32_to_avro(value: int) -> bytes:
+    return STRUCT_INT32.pack(value)
+
+
+def int64_to_avro(value: int) -> bytes:
+    return STRUCT_INT64.pack(value)
+
+
+def float_to_avro(value: float) -> bytes:
+    return STRUCT_FLOAT.pack(value)
+
+
+def double_to_avro(value: float) -> bytes:
+    return STRUCT_DOUBLE.pack(value)
+
+
+def bytes_to_avro(value: Union[bytes, str]) -> bytes:
+    if type(value) == str:
+        return value.encode()
+    else:
+        assert isinstance(value, bytes)  # appeases mypy
+        return value
+
+
+class StatsAggregator:
+    def __init__(self, type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.serialize: Any = None
+        self.trunc_lenght = trunc_length
+
+        if type_string == "BOOLEAN":
+            self.serialize = bool_to_avro
+        elif type_string == "INT32":
+            self.serialize = int32_to_avro
+        elif type_string == "INT64":
+            self.serialize = int64_to_avro
+        elif type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+        elif type_string == "FLOAT":
+            self.serialize = float_to_avro
+        elif type_string == "DOUBLE":
+            self.serialize = double_to_avro
+        elif type_string == "BYTE_ARRAY":
+            self.serialize = bytes_to_avro
+        elif type_string == "FIXED_LEN_BYTE_ARRAY":
+            self.serialize = bytes_to_avro
+        else:
+            raise AssertionError(f"Unknown physical type {type_string}")
+
+    def add_min(self, val: bytes) -> None:
+        if self.current_min is None:
+            self.current_min = val
+        else:
+            self.current_min = min(val, self.current_min)
+
+    def add_max(self, val: bytes) -> None:
+        if self.current_max is None:
+            self.current_max = val
+        else:
+            self.current_max = max(self.current_max, val)
+
+    def get_min(self) -> bytes:
+        return self.serialize(self.current_min)[: self.trunc_lenght]
+
+    def get_max(self) -> bytes:
+        return self.serialize(self.current_max)[: self.trunc_lenght]
+
+
+class MetricsMode(Enum):
+    NONE = 0
+    COUNTS = 1
+    TRUNC = 2
+    FULL = 3
+
+
+def match_metrics_mode(mode: str) -> Tuple[MetricsMode, Optional[int]]:
+    m = re.match(TRUNCATION_EXPR, mode)
+
+    if m:
+        return MetricsMode.TRUNC, int(m[1])
+    elif mode == "none":
+        return MetricsMode.NONE, None
+    elif mode == "counts":
+        return MetricsMode.COUNTS, None
+    elif mode == "full":
+        return MetricsMode.FULL, None
+    else:
+        raise AssertionError(f"Unsupported metrics mode {mode}")
+
+
+def fill_parquet_file_metadata(
+    df: DataFile,
+    metadata: pq.FileMetaData,
+    col_path_2_iceberg_id: Dict[str, int],
+    file_size: int,
+    table_metadata: Optional[TableMetadata] = None,
+) -> None:
+    """
+    Computes and fills the following fields of the DataFile object.
+
+    - file_format
+    - record_count
+    - file_size_in_bytes
+    - column_sizes
+    - value_counts
+    - null_value_counts
+    - nan_value_counts
+    - lower_bounds
+    - upper_bounds
+    - split_offsets
+
+    Args:
+        df (DataFile): A DataFile object representing the Parquet file for which metadata is to be filled.
+        metadata (pyarrow.parquet.FileMetaData): A pyarrow metadata object.
+        col_path_2_iceberg_id: A mapping of column paths as in the `path_in_schema` attribute of the colum
+            metadata to iceberg schema IDs. For scalar columns this will be the column name. For complex types
+            it could be something like `my_map.key_value.value`
+        file_size (int): The total compressed file size cannot be retrieved from the metadata and hence has to
+            be passed here. Depending on the kind of file system and pyarrow library call used, different
+            ways to obtain this value might be appropriate.
+    """
+    col_index_2_id = {}
+
+    col_names = {p.split(".")[0] for p in col_path_2_iceberg_id.keys()}
+
+    metrics_modes = {n: MetricsMode.TRUNC for n in col_names}
+    trunc_lengths: Dict[str, Optional[int]] = {n: BOUND_TRUNCATED_LENGHT for n in col_names}
+
+    if table_metadata:
+        default_mode = table_metadata.properties.get("write.metadata.metrics.default")
+
+        if default_mode:
+            m, t = match_metrics_mode(default_mode)
+
+            metrics_modes = {n: m for n in col_names}
+            trunc_lengths = {n: t for n in col_names}
+
+        for col_name in col_names:
+            col_mode = table_metadata.properties.get(f"write.metadata.metrics.column.{col_name}")

Review Comment:
   I'm afraid that this isn't correct. Let's consider the following schema:
   
   ```
   table {
     6: quux: required map<7: string, 8: map<9: string, 10: int>>
   }
   ```
   
   This has the following column names:
   
   ```python
   ['quux.value.key', 'quux.value.value', 'quux.key', 'quux.value', 'quux']
   ```
   
   But we need to set the configuration in line with the PyArrow naming convention:
   ```python
   {'quux.key_value.key': 7,
    'quux.key_value.value': 8,
    'quux.value.key_value.key': 9,
    'quux.value.key_value.value': 10,
    'qux.list.element': 5}
   ```



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1013,3 +1027,359 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+BOUND_TRUNCATED_LENGHT = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"

Review Comment:
   We can re-use the one from `Transforms.py`:
   https://github.com/apache/iceberg/blob/62cb7b5c50451085d6e59f0594292bf883af4220/python/pyiceberg/transforms.py#L100



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1013,3 +1027,359 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+BOUND_TRUNCATED_LENGHT = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+logger = logging.getLogger(__name__)
+
+# Serialization rules: https://iceberg.apache.org/spec/#binary-single-value-serialization
+#
+# Type      Binary serialization
+# boolean   0x00 for false, non-zero byte for true
+# int       Stored as 4-byte little-endian
+# long      Stored as 8-byte little-endian
+# float     Stored as 4-byte little-endian
+# double    Stored as 8-byte little-endian
+# date      Stores days from the 1970-01-01 in an 4-byte little-endian int
+# time      Stores microseconds from midnight in an 8-byte little-endian long
+# timestamp without zone	Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long
+# timestamp with zone	Stores microseconds from 1970-01-01 00:00:00.000000 UTC in an 8-byte little-endian long
+# string    UTF-8 bytes (without length)
+# uuid      16-byte big-endian value, see example in Appendix B
+# fixed(L)  Binary value
+# binary    Binary value (without length)
+#
+
+
+def bool_to_avro(value: bool) -> bytes:
+    return STRUCT_BOOL.pack(value)
+
+
+def int32_to_avro(value: int) -> bytes:
+    return STRUCT_INT32.pack(value)
+
+
+def int64_to_avro(value: int) -> bytes:
+    return STRUCT_INT64.pack(value)
+
+
+def float_to_avro(value: float) -> bytes:
+    return STRUCT_FLOAT.pack(value)
+
+
+def double_to_avro(value: float) -> bytes:
+    return STRUCT_DOUBLE.pack(value)
+
+
+def bytes_to_avro(value: Union[bytes, str]) -> bytes:
+    if type(value) == str:
+        return value.encode()
+    else:
+        assert isinstance(value, bytes)  # appeases mypy
+        return value
+
+
+class StatsAggregator:
+    def __init__(self, type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.serialize: Any = None
+        self.trunc_lenght = trunc_length
+
+        if type_string == "BOOLEAN":
+            self.serialize = bool_to_avro
+        elif type_string == "INT32":
+            self.serialize = int32_to_avro
+        elif type_string == "INT64":
+            self.serialize = int64_to_avro
+        elif type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+        elif type_string == "FLOAT":
+            self.serialize = float_to_avro
+        elif type_string == "DOUBLE":
+            self.serialize = double_to_avro
+        elif type_string == "BYTE_ARRAY":
+            self.serialize = bytes_to_avro
+        elif type_string == "FIXED_LEN_BYTE_ARRAY":
+            self.serialize = bytes_to_avro
+        else:
+            raise AssertionError(f"Unknown physical type {type_string}")
+
+    def add_min(self, val: bytes) -> None:
+        if self.current_min is None:
+            self.current_min = val
+        else:
+            self.current_min = min(val, self.current_min)
+
+    def add_max(self, val: bytes) -> None:
+        if self.current_max is None:
+            self.current_max = val
+        else:
+            self.current_max = max(self.current_max, val)
+
+    def get_min(self) -> bytes:
+        return self.serialize(self.current_min)[: self.trunc_lenght]
+
+    def get_max(self) -> bytes:
+        return self.serialize(self.current_max)[: self.trunc_lenght]
+
+
+class MetricsMode(Enum):
+    NONE = 0
+    COUNTS = 1
+    TRUNC = 2

Review Comment:
   I'm thinking, should we extend this Enum to a frozen data class annotated with a Singleton? (Similar to the Types itself?)



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1259599665


##########
python/pyiceberg/utils/file_stats.py:
##########
@@ -0,0 +1,333 @@
+#  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.
+
+import struct
+from typing import (
+    Any,
+    Dict,
+    List,
+    Union,
+)
+
+import pyarrow.lib
+import pyarrow.parquet as pq
+
+from pyiceberg.manifest import DataFile, FileFormat
+from pyiceberg.schema import Schema, SchemaVisitor, visit
+from pyiceberg.types import (
+    IcebergType,
+    ListType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StructType,
+)
+
+BOUND_TRUNCATED_LENGHT = 16

Review Comment:
   This is awesome, thank you



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] maxdebayser commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "maxdebayser (via GitHub)" <gi...@apache.org>.
maxdebayser commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1259756772


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1013,3 +1027,359 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+BOUND_TRUNCATED_LENGHT = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"

Review Comment:
   But that one is from brackets instead of parentheses :/



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1259665559


##########
python/pyiceberg/utils/file_stats.py:
##########
@@ -0,0 +1,333 @@
+#  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.
+
+import struct
+from typing import (
+    Any,
+    Dict,
+    List,
+    Union,
+)
+
+import pyarrow.lib
+import pyarrow.parquet as pq
+
+from pyiceberg.manifest import DataFile, FileFormat
+from pyiceberg.schema import Schema, SchemaVisitor, visit
+from pyiceberg.types import (
+    IcebergType,
+    ListType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StructType,
+)
+
+BOUND_TRUNCATED_LENGHT = 16
+
+# Serialization rules: https://iceberg.apache.org/spec/#binary-single-value-serialization
+#
+# Type      Binary serialization
+# boolean   0x00 for false, non-zero byte for true
+# int       Stored as 4-byte little-endian
+# long      Stored as 8-byte little-endian
+# float     Stored as 4-byte little-endian
+# double    Stored as 8-byte little-endian
+# date      Stores days from the 1970-01-01 in an 4-byte little-endian int
+# time      Stores microseconds from midnight in an 8-byte little-endian long
+# timestamp without zone	Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long
+# timestamp with zone	Stores microseconds from 1970-01-01 00:00:00.000000 UTC in an 8-byte little-endian long
+# string    UTF-8 bytes (without length)
+# uuid      16-byte big-endian value, see example in Appendix B
+# fixed(L)  Binary value
+# binary    Binary value (without length)
+#
+
+
+def bool_to_avro(value: bool) -> bytes:
+    return struct.pack("?", value)
+
+
+def int32_to_avro(value: int) -> bytes:
+    return struct.pack("<i", value)
+
+
+def int64_to_avro(value: int) -> bytes:
+    return struct.pack("<q", value)
+
+
+def float_to_avro(value: float) -> bytes:
+    return struct.pack("<f", value)
+
+
+def double_to_avro(value: float) -> bytes:
+    return struct.pack("<d", value)
+
+
+def bytes_to_avro(value: Union[bytes, str]) -> bytes:
+    if type(value) == str:
+        return value.encode()
+    else:
+        assert isinstance(value, bytes)  # appeases mypy
+        return value
+
+
+class StatsAggregator:
+    def __init__(self, type_string: str):
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.serialize: Any = None
+
+        if type_string == "BOOLEAN":
+            self.serialize = bool_to_avro
+        elif type_string == "INT32":
+            self.serialize = int32_to_avro
+        elif type_string == "INT64":
+            self.serialize = int64_to_avro
+        elif type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+        elif type_string == "FLOAT":
+            self.serialize = float_to_avro
+        elif type_string == "DOUBLE":
+            self.serialize = double_to_avro
+        elif type_string == "BYTE_ARRAY":
+            self.serialize = bytes_to_avro
+        elif type_string == "FIXED_LEN_BYTE_ARRAY":
+            self.serialize = bytes_to_avro
+        else:
+            raise AssertionError(f"Unknown physical type {type_string}")
+
+    def add_min(self, val: bytes) -> None:
+        if not self.current_min:
+            self.current_min = val
+        elif val < self.current_min:
+            self.current_min = val
+
+    def add_max(self, val: bytes) -> None:
+        if not self.current_max:
+            self.current_max = val
+        elif self.current_max < val:
+            self.current_max = val
+
+    def get_min(self) -> bytes:
+        return self.serialize(self.current_min)[:BOUND_TRUNCATED_LENGHT]
+
+    def get_max(self) -> bytes:
+        return self.serialize(self.current_max)[:BOUND_TRUNCATED_LENGHT]
+
+
+def fill_parquet_file_metadata(
+    df: DataFile, metadata: pq.FileMetaData, col_path_2_iceberg_id: Dict[str, int], file_size: int
+) -> None:
+    """
+    Computes and fills the following fields of the DataFile object.
+
+    - file_format
+    - record_count
+    - file_size_in_bytes
+    - column_sizes
+    - value_counts
+    - null_value_counts
+    - nan_value_counts
+    - lower_bounds
+    - upper_bounds
+    - split_offsets
+
+    Args:
+        df (DataFile): A DataFile object representing the Parquet file for which metadata is to be filled.
+        metadata (pyarrow.parquet.FileMetaData): A pyarrow metadata object.
+        col_path_2_iceberg_id: A mapping of column paths as in the `path_in_schema` attribute of the colum

Review Comment:
   Since Arrow is column-oriented, I'm sure that it will follow the order of the write schema:
   
   ![image](https://github.com/apache/iceberg/assets/1134248/5dd5016e-7761-4cdc-a169-d3c4744c581b)
   
   The reason I try to avoid using too many internal details from PyArrow is that we support PyArrow `[9.0.0, 12.0.1]` currently. There is no guarantee that all these internals stay the same, therefore I think we should do as much as possible within our own control (also regarding the internal Parquet types, or how the column names are structured).
   
   We already have the write schema, so we can easily filter out the primitive types:
   
   ```python
   class PyArrowStatisticsCollector(PreOrderSchemaVisitor[List[StatisticsCollector]]):
       _field_id = 0
       _schema: Schema
       _properties: Properties
   
       def __init__(self, schema: Schema, properties: Dict[str, str]):
           self._schema = schema
           self._properties = properties
   
       def schema(self, schema: Schema, struct_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
           return struct_result()
   
       def struct(self, struct: StructType, field_results: List[Callable[[], List[StatisticsCollector]]]) -> List[StatisticsCollector]:
           return list(chain(*[result() for result in field_results]))
   
       def field(self, field: NestedField, field_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
           self._field_id = field.field_id
           result = field_result()
           return result
   
       def list(self, list_type: ListType, element_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
           self._field_id = list_type.element_id
           return element_result()
   
       def map(self, map_type: MapType, key_result: Callable[[], List[StatisticsCollector]], value_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
           self._field_id = map_type.key_id
           k = key_result()
           self._field_id = map_type.value_id
           v = value_result()
           return k + v
   
       def primitive(self, primitive: PrimitiveType) -> List[StatisticsCollector]:
           return [StatisticsCollector(
               field_id=self._field_id,
               iceberg_type=primitive,
               mode=MetricsMode.TRUNC
               # schema
               # self._properties.get(f"write.metadata.metrics.column.{schema.find_column_name(self._field_id)}")
           )]
   ```
   
   This way we get a nice list of columns that we need to collect statistics for. We have:
   
   ```python
   @dataclass(frozen=True)
   class StatisticsCollector:
       field_id: int
       iceberg_type: PrimitiveType
       mode: MetricsMode
   ```
   
   Where we can use the `field_id` to properly populate the maps, and the `iceberg_type` to feed into `to_bytes` to do the conversion (so we don't have to have yet another one for PyArrow).
   
   I did a quick test, and it seems to work:
   ```python
   def test_complex_schema(table_schema_nested: Schema):
       tbl = pa.Table.from_pydict({
           "foo": ["a", "b"],
           "bar": [1, 2],
           "baz": [False, True],
           "qux": [["a", "b"], ["c", "d"]],
           "quux": [[("a", (("aa", 1), ("ab", 2)))], [("b", (("ba", 3), ("bb", 4)))]],
           "location": [[(52.377956, 4.897070), (4.897070, -122.431297)],
                        [(43.618881, -116.215019), (41.881832, -87.623177)]],
           "person": [("Fokko", 33), ("Max", 42)]  # Possible data quality issue
       },
           schema=schema_to_pyarrow(table_schema_nested)
       )
       stats_columns = pre_order_visit(table_schema_nested, PyArrowStatisticsCollector(table_schema_nested, {}))
   
       visited_paths = []
   
       def file_visitor(written_file: Any) -> None:
           visited_paths.append(written_file)
   
       with TemporaryDirectory() as tmpdir:
           pq.write_to_dataset(tbl, tmpdir, file_visitor=file_visitor)
   
       assert visited_paths[0].metadata.num_columns == len(stats_columns)
   ```



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] maxdebayser commented on pull request #7831: Compute parquet stats

Posted by "maxdebayser (via GitHub)" <gi...@apache.org>.
maxdebayser commented on PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#issuecomment-1590005818

   @Fokko, I understand your concern, I think it's because we have different use cases in mind.
   
   If I understand correctly you want to write a pyarrow.Table to a partitioned dataset with write_dataset. Therefore computing min/max on the whole Table is not what you need because you actually need the min/max for the columns of the individual files. (Just pointing out that with the metadata collector you get the stats for the row chunks, so you'll still have to compute the stats for the file from those).
   
   I'm coming from a different use case. I would like to write from Ray using something like https://docs.ray.io/en/latest/data/api/doc/ray.data.Dataset.write_parquet.html#ray.data.Dataset.write_parquet . In this case there is no global pyarrow.Table that represent the dataset, Pyarrow tables are the blocks of the dataset that each individual ray task sees, for example in `map_batches`. In this scenario the pyarrow.write_dataset cannot be used because the full dataset is not entirely loaded into the memory of any compute node. In this scenario the GIL is also not a big concern because ray uses multiple worker processes.
   
   I think we have to see if there is a way to have a single API for both use cases or if we'll need to have different API calls for both. In the second case it would be better to share part of the implementation to ensure that the behavior is consistent, but it could perhaps lead to bad performance.
   
   Regarding the efficiency, the pyarrow.compute.min function is implemented in C++, so I think the performance is probably not a huge concern here. But I can try to compare both approaches with a large enough data set to measure it.


-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1285258445


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1039,335 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.values()).union({"INT96"})
+
+
+class StatsAggregator:
+    current_min: Any
+    current_max: Any
+    trunc_length: Optional[int]
+
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min = None
+        self.current_max = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        expected_physical_type = _PRIMITIVE_TO_PHYSICAL[iceberg_type]
+        if expected_physical_type != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {expected_physical_type}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        self.current_min = val if self.current_min is None else min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        self.current_max = val if self.current_max is None else max(val, self.current_max)
+
+    def get_min(self) -> bytes:
+        return self.serialize(
+            self.current_min
+            if self.trunc_length is None
+            else TruncateTransform(width=self.trunc_length).transform(self.primitive_type)(self.current_min)
+        )
+
+    def get_max(self) -> Optional[bytes]:
+        if self.current_max is None:
+            return None
+
+        if self.primitive_type == StringType():
+            if type(self.current_max) != str:
+                raise ValueError("Expected the current_max to be a string")
+
+            s_result = self.current_max[: self.trunc_length]
+            if s_result != self.current_max:
+                chars = [*s_result]
+
+                for i in range(-1, -len(s_result) - 1, -1):
+                    try:
+                        to_inc = ord(chars[i])
+                        # will raise exception if the highest unicode code is reached
+                        _next = chr(to_inc + 1)
+                        chars[i] = _next
+                        return self.serialize("".join(chars))
+                    except ValueError:
+                        pass
+                return None  # didn't find a valid upper bound
+            return self.serialize(s_result)
+        elif self.primitive_type == BinaryType():
+            if type(self.current_max) != bytes:
+                raise ValueError("Expected the current_max to be bytes")
+            b_result = self.current_max[: self.trunc_length]
+            if b_result != self.current_max:
+                _bytes = [*b_result]
+                for i in range(-1, -len(b_result) - 1, -1):
+                    if _bytes[i] < 255:
+                        _bytes[i] += 1
+                        return b"".join([i.to_bytes(1, byteorder="little") for i in _bytes])
+                return None
+
+            return self.serialize(b_result)
+        else:
+            return self.serialize(self.current_max)[: self.trunc_length]
+
+
+DEFAULT_TRUNCATION_LENGTH = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+
+class MetricModeTypes(Enum):
+    TRUNCATE = "truncate"
+    NONE = "none"
+    COUNTS = "counts"
+    FULL = "full"
+
+
+DEFAULT_METRICS_MODE_KEY = "write.metadata.metrics.default"
+COLUMN_METRICS_MODE_KEY_PREFIX = "write.metadata.metrics.column"
+
+
+@dataclass(frozen=True)
+class MetricsMode(Singleton):
+    type: MetricModeTypes
+    length: Optional[int] = None
+
+
+def match_metrics_mode(mode: str) -> MetricsMode:
+    sanitized_mode = mode.lower()
+    if sanitized_mode.startswith("truncate"):
+        m = re.match(TRUNCATION_EXPR, mode, re.IGNORECASE)
+        if m:
+            length = int(m[1])
+            if length < 1:
+                raise ValueError("Truncation length must be larger than 0")
+            return MetricsMode(MetricModeTypes.TRUNCATE, int(m[1]))
+        else:
+            raise ValueError(f"Malformed truncate: {mode}")
+    elif sanitized_mode.startswith("none"):

Review Comment:
   Why does this permit additional characters when truncate is strict?



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1285262390


##########
python/tests/io/test_pyarrow.py:
##########
@@ -1345,3 +1374,655 @@ def test_pyarrow_wrap_fsspec(example_task: FileScanTask, table_schema_simple: Sc
 bar: [[1,2,3]]
 baz: [[true,false,null]]"""
     )
+
+
+def construct_test_table() -> Tuple[Any, Any, Union[TableMetadataV1, TableMetadataV2]]:
+    table_metadata = {
+        "format-version": 2,
+        "location": "s3://bucket/test/location",
+        "last-column-id": 7,
+        "current-schema-id": 0,
+        "schemas": [
+            {
+                "type": "struct",
+                "schema-id": 0,
+                "fields": [
+                    {"id": 1, "name": "strings", "required": False, "type": "string"},
+                    {"id": 2, "name": "floats", "required": False, "type": "float"},
+                    {
+                        "id": 3,
+                        "name": "list",
+                        "required": False,
+                        "type": {"type": "list", "element-id": 5, "element": "long", "element-required": False},
+                    },
+                    {
+                        "id": 4,
+                        "name": "maps",
+                        "required": False,
+                        "type": {
+                            "type": "map",
+                            "key-id": 6,
+                            "key": "long",
+                            "value-id": 7,
+                            "value": "long",
+                            "value-required": False,
+                        },
+                    },
+                ],
+            },
+        ],
+        "default-spec-id": 0,
+        "partition-specs": [{"spec-id": 0, "fields": []}],
+        "properties": {},
+    }
+
+    table_metadata = TableMetadataUtil.parse_obj(table_metadata)
+    arrow_schema = schema_to_pyarrow(table_metadata.schemas[0])
+
+    _strings = ["zzzzzzzzzzzzzzzzzzzz", "rrrrrrrrrrrrrrrrrrrr", None, "aaaaaaaaaaaaaaaaaaaa"]
+
+    _floats = [3.14, math.nan, 1.69, 100]
+
+    _list = [[1, 2, 3], [4, 5, 6], None, [7, 8, 9]]
+
+    _maps: List[Optional[Dict[int, int]]] = [
+        {1: 2, 3: 4},
+        None,
+        {5: 6},
+        {},
+    ]
+
+    table = pa.Table.from_pydict(
+        {
+            "strings": _strings,
+            "floats": _floats,
+            "list": _list,
+            "maps": _maps,
+        },
+        schema=arrow_schema,
+    )
+    metadata_collector: List[Any] = []
+
+    with pa.BufferOutputStream() as f:
+        with pq.ParquetWriter(f, table.schema, metadata_collector=metadata_collector) as writer:
+            writer.write_table(table)
+
+        return f.getvalue(), metadata_collector[0], table_metadata
+
+
+def test_record_count() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.record_count == 4
+
+
+def test_file_size() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.file_size_in_bytes == len(file_bytes)
+
+
+def test_value_counts() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.value_counts) == 5
+    assert datafile.value_counts[1] == 4
+    assert datafile.value_counts[2] == 4
+    assert datafile.value_counts[5] == 10  # 3 lists with 3 items and a None value
+    assert datafile.value_counts[6] == 5

Review Comment:
   It looks like both maps and lists count empty/null parent structures. I would expect this to be the number of leaf values.
   
   Can you confirm that this is also the behavior of the Java Parquet library? If not, we should just drop map and list stats because they aren't used and may not be reliable.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1288329800


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1039,344 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.values()).union({"INT96"})
+
+
+class StatsAggregator:
+    current_min: Any
+    current_max: Any
+    trunc_length: Optional[int]
+
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min = None
+        self.current_max = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        expected_physical_type = _PRIMITIVE_TO_PHYSICAL[iceberg_type]
+        if expected_physical_type != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {expected_physical_type}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def update_min(self, val: Any) -> None:
+        self.current_min = val if self.current_min is None else min(val, self.current_min)
+
+    def update_max(self, val: Any) -> None:
+        self.current_max = val if self.current_max is None else max(val, self.current_max)
+
+    def min_as_bytes(self) -> bytes:
+        return self.serialize(
+            self.current_min
+            if self.trunc_length is None
+            else TruncateTransform(width=self.trunc_length).transform(self.primitive_type)(self.current_min)
+        )
+
+    def max_as_bytes(self) -> Optional[bytes]:
+        if self.current_max is None:
+            return None
+
+        if self.primitive_type == StringType():
+            if type(self.current_max) != str:
+                raise ValueError("Expected the current_max to be a string")
+
+            s_result = self.current_max[: self.trunc_length]
+            if s_result != self.current_max:
+                chars = [*s_result]
+
+                for i in range(-1, -len(s_result) - 1, -1):
+                    try:
+                        to_inc = ord(chars[i])
+                        # will raise exception if the highest unicode code is reached
+                        _next = chr(to_inc + 1)
+                        chars[i] = _next
+                        return self.serialize("".join(chars))
+                    except ValueError:
+                        pass
+                return None  # didn't find a valid upper bound
+            return self.serialize(s_result)
+        elif self.primitive_type == BinaryType():
+            if type(self.current_max) != bytes:
+                raise ValueError("Expected the current_max to be bytes")
+            b_result = self.current_max[: self.trunc_length]
+            if b_result != self.current_max:
+                _bytes = [*b_result]
+                for i in range(-1, -len(b_result) - 1, -1):
+                    if _bytes[i] < 255:
+                        _bytes[i] += 1
+                        return b"".join([i.to_bytes(1, byteorder="little") for i in _bytes])
+                return None
+
+            return self.serialize(b_result)
+        else:
+            if self.trunc_length is not None:
+                raise ValueError(f"{self.primitive_type} cannot be truncated")
+            return self.serialize(self.current_max)
+
+
+DEFAULT_TRUNCATION_LENGTH = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+
+class MetricModeTypes(Enum):
+    TRUNCATE = "truncate"
+    NONE = "none"
+    COUNTS = "counts"
+    FULL = "full"
+
+
+DEFAULT_METRICS_MODE_KEY = "write.metadata.metrics.default"
+COLUMN_METRICS_MODE_KEY_PREFIX = "write.metadata.metrics.column"
+
+
+@dataclass(frozen=True)
+class MetricsMode(Singleton):
+    type: MetricModeTypes
+    length: Optional[int] = None
+
+
+_DEFAULT_METRICS_MODE = MetricsMode(MetricModeTypes.TRUNCATE, DEFAULT_TRUNCATION_LENGTH)
+
+
+def match_metrics_mode(mode: str) -> MetricsMode:
+    sanitized_mode = mode.lower()
+    if sanitized_mode.startswith("truncate"):
+        m = re.match(TRUNCATION_EXPR, sanitized_mode)
+        if m:
+            length = int(m[1])
+            if length < 1:
+                raise ValueError("Truncation length must be larger than 0")
+            return MetricsMode(MetricModeTypes.TRUNCATE, int(m[1]))
+        else:
+            raise ValueError(f"Malformed truncate: {mode}")
+    elif sanitized_mode.startswith("none"):
+        return MetricsMode(MetricModeTypes.NONE)
+    elif sanitized_mode.startswith("counts"):
+        return MetricsMode(MetricModeTypes.COUNTS)
+    elif sanitized_mode.startswith("full"):
+        return MetricsMode(MetricModeTypes.FULL)
+    else:
+        raise ValueError(f"Unsupported metrics mode: {mode}")
+

Review Comment:
   ```suggestion
   def match_metrics_mode(mode: str) -> MetricsMode:
       sanitized_mode = mode.trim().lower()
       if sanitized_mode.startswith("truncate"):
           m = re.match(TRUNCATION_EXPR, sanitized_mode)
           if m:
               length = int(m[1])
               if length < 1:
                   raise ValueError("Truncation length must be larger than 0")
               return MetricsMode(MetricModeTypes.TRUNCATE, int(m[1]))
           else:
               raise ValueError(f"Malformed truncate: {mode}")
       elif sanitized_mode == "none":
           return MetricsMode(MetricModeTypes.NONE)
       elif sanitized_mode == "counts":
           return MetricsMode(MetricModeTypes.COUNTS)
       elif sanitized_mode == "full":
           return MetricsMode(MetricModeTypes.FULL)
       else:
           raise ValueError(f"Unsupported metrics mode: {mode}")
   
   ```



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1039,335 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.values()).union({"INT96"})
+
+
+class StatsAggregator:
+    current_min: Any
+    current_max: Any
+    trunc_length: Optional[int]
+
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min = None
+        self.current_max = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        expected_physical_type = _PRIMITIVE_TO_PHYSICAL[iceberg_type]
+        if expected_physical_type != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {expected_physical_type}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        self.current_min = val if self.current_min is None else min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        self.current_max = val if self.current_max is None else max(val, self.current_max)
+
+    def get_min(self) -> bytes:
+        return self.serialize(
+            self.current_min
+            if self.trunc_length is None
+            else TruncateTransform(width=self.trunc_length).transform(self.primitive_type)(self.current_min)
+        )
+
+    def get_max(self) -> Optional[bytes]:
+        if self.current_max is None:
+            return None
+
+        if self.primitive_type == StringType():
+            if type(self.current_max) != str:
+                raise ValueError("Expected the current_max to be a string")
+
+            s_result = self.current_max[: self.trunc_length]
+            if s_result != self.current_max:
+                chars = [*s_result]
+
+                for i in range(-1, -len(s_result) - 1, -1):
+                    try:
+                        to_inc = ord(chars[i])
+                        # will raise exception if the highest unicode code is reached
+                        _next = chr(to_inc + 1)
+                        chars[i] = _next
+                        return self.serialize("".join(chars))
+                    except ValueError:
+                        pass
+                return None  # didn't find a valid upper bound
+            return self.serialize(s_result)
+        elif self.primitive_type == BinaryType():
+            if type(self.current_max) != bytes:
+                raise ValueError("Expected the current_max to be bytes")
+            b_result = self.current_max[: self.trunc_length]
+            if b_result != self.current_max:
+                _bytes = [*b_result]
+                for i in range(-1, -len(b_result) - 1, -1):
+                    if _bytes[i] < 255:
+                        _bytes[i] += 1
+                        return b"".join([i.to_bytes(1, byteorder="little") for i in _bytes])
+                return None
+
+            return self.serialize(b_result)
+        else:
+            return self.serialize(self.current_max)[: self.trunc_length]
+
+
+DEFAULT_TRUNCATION_LENGTH = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+
+class MetricModeTypes(Enum):
+    TRUNCATE = "truncate"
+    NONE = "none"
+    COUNTS = "counts"
+    FULL = "full"
+
+
+DEFAULT_METRICS_MODE_KEY = "write.metadata.metrics.default"
+COLUMN_METRICS_MODE_KEY_PREFIX = "write.metadata.metrics.column"
+
+
+@dataclass(frozen=True)
+class MetricsMode(Singleton):
+    type: MetricModeTypes
+    length: Optional[int] = None
+
+
+def match_metrics_mode(mode: str) -> MetricsMode:
+    sanitized_mode = mode.lower()
+    if sanitized_mode.startswith("truncate"):
+        m = re.match(TRUNCATION_EXPR, mode, re.IGNORECASE)
+        if m:
+            length = int(m[1])
+            if length < 1:
+                raise ValueError("Truncation length must be larger than 0")
+            return MetricsMode(MetricModeTypes.TRUNCATE, int(m[1]))
+        else:
+            raise ValueError(f"Malformed truncate: {mode}")
+    elif sanitized_mode.startswith("none"):
+        return MetricsMode(MetricModeTypes.NONE)
+    elif sanitized_mode.startswith("counts"):
+        return MetricsMode(MetricModeTypes.COUNTS)
+    elif sanitized_mode.startswith("full"):
+        return MetricsMode(MetricModeTypes.FULL)
+    else:
+        raise ValueError(f"Unsupported metrics mode: {mode}")
+
+
+@dataclass(frozen=True)
+class StatisticsCollector:
+    field_id: int
+    iceberg_type: PrimitiveType
+    mode: MetricsMode
+    column_name: str
+
+
+class PyArrowStatisticsCollector(PreOrderSchemaVisitor[List[StatisticsCollector]]):
+    _field_id: int = 0
+    _schema: Schema
+    _properties: Dict[str, str]
+
+    def __init__(self, schema: Schema, properties: Dict[str, str]):
+        self._schema = schema
+        self._properties = properties
+
+    def schema(self, schema: Schema, struct_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        return struct_result()
+
+    def struct(
+        self, struct: StructType, field_results: List[Callable[[], List[StatisticsCollector]]]
+    ) -> List[StatisticsCollector]:
+        return list(chain(*[result() for result in field_results]))
+
+    def field(self, field: NestedField, field_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = field.field_id

Review Comment:
   I see your point, and happy to add those fields. Currently, they are not there, so I would suggest doing that in a separate PR, to avoid going on a tangent in this PR. Created: https://github.com/apache/iceberg/issues/8273



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1039,335 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.values()).union({"INT96"})
+
+
+class StatsAggregator:
+    current_min: Any
+    current_max: Any
+    trunc_length: Optional[int]
+
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min = None
+        self.current_max = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        expected_physical_type = _PRIMITIVE_TO_PHYSICAL[iceberg_type]
+        if expected_physical_type != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {expected_physical_type}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        self.current_min = val if self.current_min is None else min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        self.current_max = val if self.current_max is None else max(val, self.current_max)
+
+    def get_min(self) -> bytes:
+        return self.serialize(
+            self.current_min
+            if self.trunc_length is None
+            else TruncateTransform(width=self.trunc_length).transform(self.primitive_type)(self.current_min)
+        )
+
+    def get_max(self) -> Optional[bytes]:
+        if self.current_max is None:
+            return None
+
+        if self.primitive_type == StringType():
+            if type(self.current_max) != str:
+                raise ValueError("Expected the current_max to be a string")
+
+            s_result = self.current_max[: self.trunc_length]
+            if s_result != self.current_max:
+                chars = [*s_result]
+
+                for i in range(-1, -len(s_result) - 1, -1):
+                    try:
+                        to_inc = ord(chars[i])
+                        # will raise exception if the highest unicode code is reached
+                        _next = chr(to_inc + 1)
+                        chars[i] = _next
+                        return self.serialize("".join(chars))
+                    except ValueError:
+                        pass
+                return None  # didn't find a valid upper bound
+            return self.serialize(s_result)
+        elif self.primitive_type == BinaryType():
+            if type(self.current_max) != bytes:
+                raise ValueError("Expected the current_max to be bytes")
+            b_result = self.current_max[: self.trunc_length]
+            if b_result != self.current_max:
+                _bytes = [*b_result]
+                for i in range(-1, -len(b_result) - 1, -1):
+                    if _bytes[i] < 255:
+                        _bytes[i] += 1
+                        return b"".join([i.to_bytes(1, byteorder="little") for i in _bytes])
+                return None
+
+            return self.serialize(b_result)
+        else:
+            return self.serialize(self.current_max)[: self.trunc_length]
+
+
+DEFAULT_TRUNCATION_LENGTH = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+
+class MetricModeTypes(Enum):
+    TRUNCATE = "truncate"
+    NONE = "none"
+    COUNTS = "counts"
+    FULL = "full"
+
+
+DEFAULT_METRICS_MODE_KEY = "write.metadata.metrics.default"
+COLUMN_METRICS_MODE_KEY_PREFIX = "write.metadata.metrics.column"
+
+
+@dataclass(frozen=True)
+class MetricsMode(Singleton):
+    type: MetricModeTypes
+    length: Optional[int] = None
+
+
+def match_metrics_mode(mode: str) -> MetricsMode:
+    sanitized_mode = mode.lower()
+    if sanitized_mode.startswith("truncate"):
+        m = re.match(TRUNCATION_EXPR, mode, re.IGNORECASE)
+        if m:
+            length = int(m[1])
+            if length < 1:
+                raise ValueError("Truncation length must be larger than 0")
+            return MetricsMode(MetricModeTypes.TRUNCATE, int(m[1]))
+        else:
+            raise ValueError(f"Malformed truncate: {mode}")
+    elif sanitized_mode.startswith("none"):
+        return MetricsMode(MetricModeTypes.NONE)
+    elif sanitized_mode.startswith("counts"):
+        return MetricsMode(MetricModeTypes.COUNTS)
+    elif sanitized_mode.startswith("full"):
+        return MetricsMode(MetricModeTypes.FULL)
+    else:
+        raise ValueError(f"Unsupported metrics mode: {mode}")
+
+
+@dataclass(frozen=True)
+class StatisticsCollector:
+    field_id: int
+    iceberg_type: PrimitiveType
+    mode: MetricsMode
+    column_name: str
+
+
+class PyArrowStatisticsCollector(PreOrderSchemaVisitor[List[StatisticsCollector]]):
+    _field_id: int = 0
+    _schema: Schema
+    _properties: Dict[str, str]
+
+    def __init__(self, schema: Schema, properties: Dict[str, str]):
+        self._schema = schema
+        self._properties = properties
+
+    def schema(self, schema: Schema, struct_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        return struct_result()
+
+    def struct(
+        self, struct: StructType, field_results: List[Callable[[], List[StatisticsCollector]]]
+    ) -> List[StatisticsCollector]:
+        return list(chain(*[result() for result in field_results]))
+
+    def field(self, field: NestedField, field_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = field.field_id
+        result = field_result()
+        return result
+
+    def list(self, list_type: ListType, element_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = list_type.element_id
+        return element_result()
+
+    def map(
+        self,
+        map_type: MapType,
+        key_result: Callable[[], List[StatisticsCollector]],
+        value_result: Callable[[], List[StatisticsCollector]],
+    ) -> List[StatisticsCollector]:
+        self._field_id = map_type.key_id
+        k = key_result()
+        self._field_id = map_type.value_id
+        v = value_result()
+        return k + v
+
+    def primitive(self, primitive: PrimitiveType) -> List[StatisticsCollector]:
+        column_name = self._schema.find_column_name(self._field_id)
+        if column_name is None:
+            raise ValueError(f"Column for field {self._field_id} not found")
+
+        metrics_mode = MetricsMode(MetricModeTypes.TRUNCATE, DEFAULT_TRUNCATION_LENGTH)
+
+        default_mode = self._properties.get(DEFAULT_METRICS_MODE_KEY)
+        if default_mode:
+            metrics_mode = match_metrics_mode(default_mode)
+
+        col_mode = self._properties.get(f"{COLUMN_METRICS_MODE_KEY_PREFIX}.{column_name}")
+        if col_mode:
+            metrics_mode = match_metrics_mode(col_mode)
+
+        if (
+            not (isinstance(primitive, StringType) or isinstance(primitive, BinaryType))
+            and metrics_mode.type == MetricModeTypes.TRUNCATE
+        ):
+            metrics_mode = MetricsMode(MetricModeTypes.FULL)
+
+        return [StatisticsCollector(field_id=self._field_id, iceberg_type=primitive, mode=metrics_mode, column_name=column_name)]
+
+
+def fill_parquet_file_metadata(
+    df: DataFile,
+    parquet_metadata: pq.FileMetaData,
+    file_size: int,
+    table_metadata: TableMetadata,

Review Comment:
   I agree that we shouldn't pass in the `TableMetadata` here. Since we know the write schema, we can just pass that one in.
   
   > Metrics modes can be pre-calculated to produce a map from field ID to mode. It makes more sense to do this in a common util method than to mix the code into the already complex StatisticsCollector visitor
   
   I'm not sure about this one, we know the order of the columns, so we can just rely on that one. We can also have a lookup, but I'm not sure that it makes it much simpler than keeping the mode next to the id and type.



##########
python/tests/io/test_pyarrow.py:
##########
@@ -1345,3 +1374,655 @@ def test_pyarrow_wrap_fsspec(example_task: FileScanTask, table_schema_simple: Sc
 bar: [[1,2,3]]
 baz: [[true,false,null]]"""
     )
+
+
+def construct_test_table() -> Tuple[Any, Any, Union[TableMetadataV1, TableMetadataV2]]:

Review Comment:
   `test_pyarrow_stats.py`?



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1039,335 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.values()).union({"INT96"})
+
+
+class StatsAggregator:
+    current_min: Any
+    current_max: Any
+    trunc_length: Optional[int]
+
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min = None
+        self.current_max = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        expected_physical_type = _PRIMITIVE_TO_PHYSICAL[iceberg_type]
+        if expected_physical_type != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {expected_physical_type}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        self.current_min = val if self.current_min is None else min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        self.current_max = val if self.current_max is None else max(val, self.current_max)
+
+    def get_min(self) -> bytes:
+        return self.serialize(
+            self.current_min
+            if self.trunc_length is None
+            else TruncateTransform(width=self.trunc_length).transform(self.primitive_type)(self.current_min)
+        )
+
+    def get_max(self) -> Optional[bytes]:
+        if self.current_max is None:
+            return None
+
+        if self.primitive_type == StringType():
+            if type(self.current_max) != str:
+                raise ValueError("Expected the current_max to be a string")
+
+            s_result = self.current_max[: self.trunc_length]
+            if s_result != self.current_max:
+                chars = [*s_result]
+
+                for i in range(-1, -len(s_result) - 1, -1):
+                    try:
+                        to_inc = ord(chars[i])
+                        # will raise exception if the highest unicode code is reached
+                        _next = chr(to_inc + 1)
+                        chars[i] = _next
+                        return self.serialize("".join(chars))
+                    except ValueError:
+                        pass
+                return None  # didn't find a valid upper bound
+            return self.serialize(s_result)
+        elif self.primitive_type == BinaryType():
+            if type(self.current_max) != bytes:
+                raise ValueError("Expected the current_max to be bytes")
+            b_result = self.current_max[: self.trunc_length]
+            if b_result != self.current_max:
+                _bytes = [*b_result]
+                for i in range(-1, -len(b_result) - 1, -1):
+                    if _bytes[i] < 255:
+                        _bytes[i] += 1
+                        return b"".join([i.to_bytes(1, byteorder="little") for i in _bytes])
+                return None
+
+            return self.serialize(b_result)
+        else:
+            return self.serialize(self.current_max)[: self.trunc_length]
+
+
+DEFAULT_TRUNCATION_LENGTH = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+
+class MetricModeTypes(Enum):
+    TRUNCATE = "truncate"
+    NONE = "none"
+    COUNTS = "counts"
+    FULL = "full"
+
+
+DEFAULT_METRICS_MODE_KEY = "write.metadata.metrics.default"
+COLUMN_METRICS_MODE_KEY_PREFIX = "write.metadata.metrics.column"
+
+
+@dataclass(frozen=True)
+class MetricsMode(Singleton):
+    type: MetricModeTypes
+    length: Optional[int] = None
+
+
+def match_metrics_mode(mode: str) -> MetricsMode:
+    sanitized_mode = mode.lower()
+    if sanitized_mode.startswith("truncate"):
+        m = re.match(TRUNCATION_EXPR, mode, re.IGNORECASE)
+        if m:
+            length = int(m[1])
+            if length < 1:
+                raise ValueError("Truncation length must be larger than 0")
+            return MetricsMode(MetricModeTypes.TRUNCATE, int(m[1]))
+        else:
+            raise ValueError(f"Malformed truncate: {mode}")
+    elif sanitized_mode.startswith("none"):
+        return MetricsMode(MetricModeTypes.NONE)
+    elif sanitized_mode.startswith("counts"):
+        return MetricsMode(MetricModeTypes.COUNTS)
+    elif sanitized_mode.startswith("full"):
+        return MetricsMode(MetricModeTypes.FULL)
+    else:
+        raise ValueError(f"Unsupported metrics mode: {mode}")
+
+
+@dataclass(frozen=True)
+class StatisticsCollector:
+    field_id: int
+    iceberg_type: PrimitiveType
+    mode: MetricsMode
+    column_name: str
+
+
+class PyArrowStatisticsCollector(PreOrderSchemaVisitor[List[StatisticsCollector]]):
+    _field_id: int = 0
+    _schema: Schema
+    _properties: Dict[str, str]
+
+    def __init__(self, schema: Schema, properties: Dict[str, str]):
+        self._schema = schema
+        self._properties = properties
+
+    def schema(self, schema: Schema, struct_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        return struct_result()
+
+    def struct(
+        self, struct: StructType, field_results: List[Callable[[], List[StatisticsCollector]]]
+    ) -> List[StatisticsCollector]:
+        return list(chain(*[result() for result in field_results]))
+
+    def field(self, field: NestedField, field_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = field.field_id
+        result = field_result()
+        return result
+
+    def list(self, list_type: ListType, element_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = list_type.element_id
+        return element_result()
+
+    def map(
+        self,
+        map_type: MapType,
+        key_result: Callable[[], List[StatisticsCollector]],
+        value_result: Callable[[], List[StatisticsCollector]],
+    ) -> List[StatisticsCollector]:
+        self._field_id = map_type.key_id
+        k = key_result()
+        self._field_id = map_type.value_id
+        v = value_result()
+        return k + v
+
+    def primitive(self, primitive: PrimitiveType) -> List[StatisticsCollector]:
+        column_name = self._schema.find_column_name(self._field_id)
+        if column_name is None:
+            raise ValueError(f"Column for field {self._field_id} not found")
+
+        metrics_mode = MetricsMode(MetricModeTypes.TRUNCATE, DEFAULT_TRUNCATION_LENGTH)
+
+        default_mode = self._properties.get(DEFAULT_METRICS_MODE_KEY)
+        if default_mode:
+            metrics_mode = match_metrics_mode(default_mode)
+
+        col_mode = self._properties.get(f"{COLUMN_METRICS_MODE_KEY_PREFIX}.{column_name}")
+        if col_mode:
+            metrics_mode = match_metrics_mode(col_mode)
+
+        if (
+            not (isinstance(primitive, StringType) or isinstance(primitive, BinaryType))
+            and metrics_mode.type == MetricModeTypes.TRUNCATE
+        ):
+            metrics_mode = MetricsMode(MetricModeTypes.FULL)
+
+        return [StatisticsCollector(field_id=self._field_id, iceberg_type=primitive, mode=metrics_mode, column_name=column_name)]
+
+
+def fill_parquet_file_metadata(
+    df: DataFile,
+    parquet_metadata: pq.FileMetaData,
+    file_size: int,
+    table_metadata: TableMetadata,
+) -> None:
+    """
+    Computes and fills the following fields of the DataFile object.
+
+    - file_format
+    - record_count
+    - file_size_in_bytes
+    - column_sizes
+    - value_counts
+    - null_value_counts
+    - nan_value_counts
+    - lower_bounds
+    - upper_bounds
+    - split_offsets
+
+    Args:
+        df (DataFile): A DataFile object representing the Parquet file for which metadata is to be filled.
+        parquet_metadata (pyarrow.parquet.FileMetaData): A pyarrow metadata object.
+        file_size (int): The total compressed file size cannot be retrieved from the metadata and hence has to
+            be passed here. Depending on the kind of file system and pyarrow library call used, different
+            ways to obtain this value might be appropriate.
+        table_metadata (pyiceberg.table.metadata.TableMetadata): The Iceberg table metadata. It is required to
+            compute the mapping if column position to iceberg schema type id. It's also used to set the mode
+            for column metrics collection
+    """
+    schema = next(filter(lambda s: s.schema_id == table_metadata.current_schema_id, table_metadata.schemas))
+
+    stats_columns = pre_order_visit(schema, PyArrowStatisticsCollector(schema, table_metadata.properties))
+
+    if parquet_metadata.num_columns != len(stats_columns):
+        raise ValueError(
+            f"Number of columns in metadata ({len(stats_columns)}) is different from the number of columns in pyarrow table ({parquet_metadata.num_columns})"
+        )
+
+    column_sizes: Dict[int, int] = {}
+    value_counts: Dict[int, int] = {}
+    split_offsets: List[int] = []
+
+    null_value_counts: Dict[int, int] = {}
+    nan_value_counts: Dict[int, int] = {}
+
+    col_aggs = {}
+
+    for r in range(parquet_metadata.num_row_groups):
+        # References:
+        # https://github.com/apache/iceberg/blob/fc381a81a1fdb8f51a0637ca27cd30673bd7aad3/parquet/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java#L232
+        # https://github.com/apache/parquet-mr/blob/ac29db4611f86a07cc6877b416aa4b183e09b353/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/ColumnChunkMetaData.java#L184
+
+        row_group = parquet_metadata.row_group(r)
+
+        data_offset = row_group.column(0).data_page_offset
+        dictionary_offset = row_group.column(0).dictionary_page_offset
+
+        if row_group.column(0).has_dictionary_page and dictionary_offset < data_offset:
+            split_offsets.append(dictionary_offset)
+        else:
+            split_offsets.append(data_offset)
+
+        for pos, stats_col in enumerate(stats_columns):
+            field_id = stats_col.field_id
+
+            column = row_group.column(pos)
+
+            column_sizes[field_id] = column_sizes.get(field_id, 0) + column.total_compressed_size
+
+            if stats_col.mode == MetricsMode(MetricModeTypes.NONE):
+                continue
+
+            value_counts[field_id] = value_counts.get(field_id, 0) + column.num_values
+
+            if column.is_stats_set:
+                try:
+                    statistics = column.statistics
+
+                    null_value_counts[field_id] = null_value_counts.get(field_id, 0) + statistics.null_count
+
+                    if stats_col.mode == MetricsMode(MetricModeTypes.COUNTS):
+                        continue
+
+                    if field_id not in col_aggs:
+                        col_aggs[field_id] = StatsAggregator(
+                            stats_col.iceberg_type, statistics.physical_type, stats_col.mode.length
+                        )
+
+                    col_aggs[field_id].add_min(statistics.min)
+                    col_aggs[field_id].add_max(statistics.max)
+
+                except pyarrow.lib.ArrowNotImplementedError as e:
+                    logger.warning(e)
+            else:
+                logger.warning("PyArrow statistics missing for column %d when writing file", pos)
+
+    split_offsets.sort()
+
+    lower_bounds = {}
+    upper_bounds = {}
+
+    for k, agg in col_aggs.items():
+        lower_bounds[k] = agg.get_min()
+        _max = agg.get_max()
+        if _max is not None:
+            upper_bounds[k] = _max
+
+    df.file_format = FileFormat.PARQUET

Review Comment:
   No, I prefer to keep them immutable, but I'm okay with it for now



##########
python/tests/io/test_pyarrow.py:
##########
@@ -1345,3 +1374,655 @@ def test_pyarrow_wrap_fsspec(example_task: FileScanTask, table_schema_simple: Sc
 bar: [[1,2,3]]
 baz: [[true,false,null]]"""
     )
+
+
+def construct_test_table() -> Tuple[Any, Any, Union[TableMetadataV1, TableMetadataV2]]:
+    table_metadata = {
+        "format-version": 2,
+        "location": "s3://bucket/test/location",
+        "last-column-id": 7,
+        "current-schema-id": 0,
+        "schemas": [
+            {
+                "type": "struct",
+                "schema-id": 0,
+                "fields": [
+                    {"id": 1, "name": "strings", "required": False, "type": "string"},
+                    {"id": 2, "name": "floats", "required": False, "type": "float"},
+                    {
+                        "id": 3,
+                        "name": "list",
+                        "required": False,
+                        "type": {"type": "list", "element-id": 5, "element": "long", "element-required": False},
+                    },
+                    {
+                        "id": 4,
+                        "name": "maps",
+                        "required": False,
+                        "type": {
+                            "type": "map",
+                            "key-id": 6,
+                            "key": "long",
+                            "value-id": 7,
+                            "value": "long",
+                            "value-required": False,
+                        },
+                    },
+                ],
+            },
+        ],
+        "default-spec-id": 0,
+        "partition-specs": [{"spec-id": 0, "fields": []}],
+        "properties": {},
+    }
+
+    table_metadata = TableMetadataUtil.parse_obj(table_metadata)
+    arrow_schema = schema_to_pyarrow(table_metadata.schemas[0])
+
+    _strings = ["zzzzzzzzzzzzzzzzzzzz", "rrrrrrrrrrrrrrrrrrrr", None, "aaaaaaaaaaaaaaaaaaaa"]
+
+    _floats = [3.14, math.nan, 1.69, 100]
+
+    _list = [[1, 2, 3], [4, 5, 6], None, [7, 8, 9]]
+
+    _maps: List[Optional[Dict[int, int]]] = [
+        {1: 2, 3: 4},
+        None,
+        {5: 6},
+        {},
+    ]
+
+    table = pa.Table.from_pydict(
+        {
+            "strings": _strings,
+            "floats": _floats,
+            "list": _list,
+            "maps": _maps,
+        },
+        schema=arrow_schema,
+    )
+    metadata_collector: List[Any] = []
+
+    with pa.BufferOutputStream() as f:
+        with pq.ParquetWriter(f, table.schema, metadata_collector=metadata_collector) as writer:
+            writer.write_table(table)
+
+        return f.getvalue(), metadata_collector[0], table_metadata
+
+
+def test_record_count() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.record_count == 4
+
+
+def test_file_size() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.file_size_in_bytes == len(file_bytes)
+
+
+def test_value_counts() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.value_counts) == 5
+    assert datafile.value_counts[1] == 4
+    assert datafile.value_counts[2] == 4
+    assert datafile.value_counts[5] == 10  # 3 lists with 3 items and a None value
+    assert datafile.value_counts[6] == 5
+    assert datafile.value_counts[7] == 5
+
+
+def test_column_sizes() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.column_sizes) == 5
+    # these values are an artifact of how the write_table encodes the columns
+    assert datafile.column_sizes[1] == 116
+    assert datafile.column_sizes[2] == 89
+    assert datafile.column_sizes[5] == 151
+    assert datafile.column_sizes[6] == 117
+    assert datafile.column_sizes[7] == 117
+
+
+def test_null_and_nan_counts() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.null_value_counts) == 5
+    assert datafile.null_value_counts[1] == 1
+    assert datafile.null_value_counts[2] == 0
+    assert datafile.null_value_counts[5] == 1

Review Comment:
   Type:
   ```json
   {"type": "list", "element-id": 5, "element": "long", "element-required": False}
   ```
   And the data:
   ```python
   _list = [[1, 2, 3], [4, 5, 6], None, [7, 8, 9]]
   ```
   I count a single `null`



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1039,344 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.values()).union({"INT96"})
+
+
+class StatsAggregator:
+    current_min: Any
+    current_max: Any
+    trunc_length: Optional[int]
+
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min = None
+        self.current_max = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        expected_physical_type = _PRIMITIVE_TO_PHYSICAL[iceberg_type]
+        if expected_physical_type != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {expected_physical_type}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def update_min(self, val: Any) -> None:
+        self.current_min = val if self.current_min is None else min(val, self.current_min)
+
+    def update_max(self, val: Any) -> None:
+        self.current_max = val if self.current_max is None else max(val, self.current_max)
+
+    def min_as_bytes(self) -> bytes:
+        return self.serialize(
+            self.current_min
+            if self.trunc_length is None
+            else TruncateTransform(width=self.trunc_length).transform(self.primitive_type)(self.current_min)
+        )
+
+    def max_as_bytes(self) -> Optional[bytes]:
+        if self.current_max is None:
+            return None
+
+        if self.primitive_type == StringType():
+            if type(self.current_max) != str:
+                raise ValueError("Expected the current_max to be a string")
+
+            s_result = self.current_max[: self.trunc_length]
+            if s_result != self.current_max:
+                chars = [*s_result]
+
+                for i in range(-1, -len(s_result) - 1, -1):
+                    try:
+                        to_inc = ord(chars[i])
+                        # will raise exception if the highest unicode code is reached
+                        _next = chr(to_inc + 1)
+                        chars[i] = _next
+                        return self.serialize("".join(chars))
+                    except ValueError:
+                        pass
+                return None  # didn't find a valid upper bound
+            return self.serialize(s_result)
+        elif self.primitive_type == BinaryType():
+            if type(self.current_max) != bytes:
+                raise ValueError("Expected the current_max to be bytes")
+            b_result = self.current_max[: self.trunc_length]
+            if b_result != self.current_max:
+                _bytes = [*b_result]
+                for i in range(-1, -len(b_result) - 1, -1):
+                    if _bytes[i] < 255:
+                        _bytes[i] += 1
+                        return b"".join([i.to_bytes(1, byteorder="little") for i in _bytes])
+                return None
+
+            return self.serialize(b_result)
+        else:
+            if self.trunc_length is not None:
+                raise ValueError(f"{self.primitive_type} cannot be truncated")
+            return self.serialize(self.current_max)
+
+
+DEFAULT_TRUNCATION_LENGTH = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+
+class MetricModeTypes(Enum):
+    TRUNCATE = "truncate"
+    NONE = "none"
+    COUNTS = "counts"
+    FULL = "full"
+
+
+DEFAULT_METRICS_MODE_KEY = "write.metadata.metrics.default"
+COLUMN_METRICS_MODE_KEY_PREFIX = "write.metadata.metrics.column"
+
+
+@dataclass(frozen=True)
+class MetricsMode(Singleton):
+    type: MetricModeTypes
+    length: Optional[int] = None
+
+
+_DEFAULT_METRICS_MODE = MetricsMode(MetricModeTypes.TRUNCATE, DEFAULT_TRUNCATION_LENGTH)
+
+
+def match_metrics_mode(mode: str) -> MetricsMode:
+    sanitized_mode = mode.lower()
+    if sanitized_mode.startswith("truncate"):
+        m = re.match(TRUNCATION_EXPR, sanitized_mode)
+        if m:
+            length = int(m[1])
+            if length < 1:
+                raise ValueError("Truncation length must be larger than 0")
+            return MetricsMode(MetricModeTypes.TRUNCATE, int(m[1]))
+        else:
+            raise ValueError(f"Malformed truncate: {mode}")
+    elif sanitized_mode.startswith("none"):
+        return MetricsMode(MetricModeTypes.NONE)
+    elif sanitized_mode.startswith("counts"):
+        return MetricsMode(MetricModeTypes.COUNTS)
+    elif sanitized_mode.startswith("full"):
+        return MetricsMode(MetricModeTypes.FULL)
+    else:
+        raise ValueError(f"Unsupported metrics mode: {mode}")
+
+
+@dataclass(frozen=True)
+class StatisticsCollector:
+    field_id: int
+    iceberg_type: PrimitiveType
+    mode: MetricsMode
+    column_name: str
+
+
+class PyArrowStatisticsCollector(PreOrderSchemaVisitor[List[StatisticsCollector]]):
+    _field_id: int = 0
+    _schema: Schema
+    _properties: Dict[str, str]
+    _default_mode: Optional[str]
+
+    def __init__(self, schema: Schema, properties: Dict[str, str]):
+        self._schema = schema
+        self._properties = properties
+        self._default_mode = self._properties.get(DEFAULT_METRICS_MODE_KEY)
+
+    def schema(self, schema: Schema, struct_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        return struct_result()
+
+    def struct(
+        self, struct: StructType, field_results: List[Callable[[], List[StatisticsCollector]]]
+    ) -> List[StatisticsCollector]:
+        return list(chain(*[result() for result in field_results]))
+
+    def field(self, field: NestedField, field_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = field.field_id
+        result = field_result()
+        return result
+
+    def list(self, list_type: ListType, element_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = list_type.element_id
+        return element_result()
+
+    def map(
+        self,
+        map_type: MapType,
+        key_result: Callable[[], List[StatisticsCollector]],
+        value_result: Callable[[], List[StatisticsCollector]],
+    ) -> List[StatisticsCollector]:
+        self._field_id = map_type.key_id
+        k = key_result()
+        self._field_id = map_type.value_id
+        v = value_result()
+        return k + v
+
+    def primitive(self, primitive: PrimitiveType) -> List[StatisticsCollector]:
+        column_name = self._schema.find_column_name(self._field_id)
+        if column_name is None:
+            return []
+
+        metrics_mode = _DEFAULT_METRICS_MODE
+
+        if self._default_mode:
+            metrics_mode = match_metrics_mode(self._default_mode)
+
+        col_mode = self._properties.get(f"{COLUMN_METRICS_MODE_KEY_PREFIX}.{column_name}")
+        if col_mode:
+            metrics_mode = match_metrics_mode(col_mode)
+
+        if (
+            not (isinstance(primitive, StringType) or isinstance(primitive, BinaryType))
+            and metrics_mode.type == MetricModeTypes.TRUNCATE
+        ):
+            metrics_mode = MetricsMode(MetricModeTypes.FULL)
+
+        return [StatisticsCollector(field_id=self._field_id, iceberg_type=primitive, mode=metrics_mode, column_name=column_name)]
+
+
+def fill_parquet_file_metadata(

Review Comment:
   Let's make sure that this isn't being used outside of the module.
   ```suggestion
   def _fill_parquet_file_metadata(
   ```



##########
python/tests/io/test_pyarrow.py:
##########
@@ -1345,3 +1374,655 @@ def test_pyarrow_wrap_fsspec(example_task: FileScanTask, table_schema_simple: Sc
 bar: [[1,2,3]]
 baz: [[true,false,null]]"""
     )
+
+
+def construct_test_table() -> Tuple[Any, Any, Union[TableMetadataV1, TableMetadataV2]]:
+    table_metadata = {
+        "format-version": 2,
+        "location": "s3://bucket/test/location",
+        "last-column-id": 7,
+        "current-schema-id": 0,
+        "schemas": [
+            {
+                "type": "struct",
+                "schema-id": 0,
+                "fields": [
+                    {"id": 1, "name": "strings", "required": False, "type": "string"},
+                    {"id": 2, "name": "floats", "required": False, "type": "float"},
+                    {
+                        "id": 3,
+                        "name": "list",
+                        "required": False,
+                        "type": {"type": "list", "element-id": 5, "element": "long", "element-required": False},
+                    },
+                    {
+                        "id": 4,
+                        "name": "maps",
+                        "required": False,
+                        "type": {
+                            "type": "map",
+                            "key-id": 6,
+                            "key": "long",
+                            "value-id": 7,
+                            "value": "long",
+                            "value-required": False,
+                        },
+                    },
+                ],
+            },
+        ],
+        "default-spec-id": 0,
+        "partition-specs": [{"spec-id": 0, "fields": []}],
+        "properties": {},
+    }
+
+    table_metadata = TableMetadataUtil.parse_obj(table_metadata)
+    arrow_schema = schema_to_pyarrow(table_metadata.schemas[0])
+
+    _strings = ["zzzzzzzzzzzzzzzzzzzz", "rrrrrrrrrrrrrrrrrrrr", None, "aaaaaaaaaaaaaaaaaaaa"]
+
+    _floats = [3.14, math.nan, 1.69, 100]
+
+    _list = [[1, 2, 3], [4, 5, 6], None, [7, 8, 9]]
+
+    _maps: List[Optional[Dict[int, int]]] = [
+        {1: 2, 3: 4},
+        None,
+        {5: 6},
+        {},
+    ]
+
+    table = pa.Table.from_pydict(
+        {
+            "strings": _strings,
+            "floats": _floats,
+            "list": _list,
+            "maps": _maps,
+        },
+        schema=arrow_schema,
+    )
+    metadata_collector: List[Any] = []
+
+    with pa.BufferOutputStream() as f:
+        with pq.ParquetWriter(f, table.schema, metadata_collector=metadata_collector) as writer:
+            writer.write_table(table)
+
+        return f.getvalue(), metadata_collector[0], table_metadata
+
+
+def test_record_count() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.record_count == 4
+
+
+def test_file_size() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.file_size_in_bytes == len(file_bytes)
+
+
+def test_value_counts() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.value_counts) == 5
+    assert datafile.value_counts[1] == 4
+    assert datafile.value_counts[2] == 4
+    assert datafile.value_counts[5] == 10  # 3 lists with 3 items and a None value
+    assert datafile.value_counts[6] == 5
+    assert datafile.value_counts[7] == 5
+
+
+def test_column_sizes() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.column_sizes) == 5
+    # these values are an artifact of how the write_table encodes the columns
+    assert datafile.column_sizes[1] == 116
+    assert datafile.column_sizes[2] == 89
+    assert datafile.column_sizes[5] == 151
+    assert datafile.column_sizes[6] == 117
+    assert datafile.column_sizes[7] == 117
+
+
+def test_null_and_nan_counts() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.null_value_counts) == 5
+    assert datafile.null_value_counts[1] == 1
+    assert datafile.null_value_counts[2] == 0
+    assert datafile.null_value_counts[5] == 1
+    assert datafile.null_value_counts[6] == 2
+    assert datafile.null_value_counts[7] == 2
+
+    # #arrow does not include this in the statistics
+    # assert len(datafile.nan_value_counts)  == 3
+    # assert datafile.nan_value_counts[1]    == 0

Review Comment:
   I was just wondering. The evaluator is not taking those into consideration for any not `float` or `double` type, any reason to still add those?



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1039,335 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.values()).union({"INT96"})
+
+
+class StatsAggregator:
+    current_min: Any
+    current_max: Any
+    trunc_length: Optional[int]
+
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min = None
+        self.current_max = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        expected_physical_type = _PRIMITIVE_TO_PHYSICAL[iceberg_type]
+        if expected_physical_type != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {expected_physical_type}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        self.current_min = val if self.current_min is None else min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        self.current_max = val if self.current_max is None else max(val, self.current_max)
+
+    def get_min(self) -> bytes:
+        return self.serialize(
+            self.current_min
+            if self.trunc_length is None
+            else TruncateTransform(width=self.trunc_length).transform(self.primitive_type)(self.current_min)
+        )
+
+    def get_max(self) -> Optional[bytes]:
+        if self.current_max is None:
+            return None
+
+        if self.primitive_type == StringType():
+            if type(self.current_max) != str:
+                raise ValueError("Expected the current_max to be a string")
+
+            s_result = self.current_max[: self.trunc_length]
+            if s_result != self.current_max:
+                chars = [*s_result]
+
+                for i in range(-1, -len(s_result) - 1, -1):
+                    try:
+                        to_inc = ord(chars[i])
+                        # will raise exception if the highest unicode code is reached
+                        _next = chr(to_inc + 1)
+                        chars[i] = _next
+                        return self.serialize("".join(chars))
+                    except ValueError:
+                        pass
+                return None  # didn't find a valid upper bound
+            return self.serialize(s_result)
+        elif self.primitive_type == BinaryType():
+            if type(self.current_max) != bytes:
+                raise ValueError("Expected the current_max to be bytes")
+            b_result = self.current_max[: self.trunc_length]
+            if b_result != self.current_max:
+                _bytes = [*b_result]
+                for i in range(-1, -len(b_result) - 1, -1):
+                    if _bytes[i] < 255:
+                        _bytes[i] += 1
+                        return b"".join([i.to_bytes(1, byteorder="little") for i in _bytes])
+                return None
+
+            return self.serialize(b_result)
+        else:
+            return self.serialize(self.current_max)[: self.trunc_length]
+
+
+DEFAULT_TRUNCATION_LENGTH = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+
+class MetricModeTypes(Enum):
+    TRUNCATE = "truncate"
+    NONE = "none"
+    COUNTS = "counts"
+    FULL = "full"
+
+
+DEFAULT_METRICS_MODE_KEY = "write.metadata.metrics.default"
+COLUMN_METRICS_MODE_KEY_PREFIX = "write.metadata.metrics.column"
+
+
+@dataclass(frozen=True)
+class MetricsMode(Singleton):
+    type: MetricModeTypes
+    length: Optional[int] = None
+
+
+def match_metrics_mode(mode: str) -> MetricsMode:
+    sanitized_mode = mode.lower()
+    if sanitized_mode.startswith("truncate"):
+        m = re.match(TRUNCATION_EXPR, mode, re.IGNORECASE)
+        if m:
+            length = int(m[1])
+            if length < 1:
+                raise ValueError("Truncation length must be larger than 0")
+            return MetricsMode(MetricModeTypes.TRUNCATE, int(m[1]))
+        else:
+            raise ValueError(f"Malformed truncate: {mode}")
+    elif sanitized_mode.startswith("none"):
+        return MetricsMode(MetricModeTypes.NONE)
+    elif sanitized_mode.startswith("counts"):
+        return MetricsMode(MetricModeTypes.COUNTS)
+    elif sanitized_mode.startswith("full"):
+        return MetricsMode(MetricModeTypes.FULL)
+    else:
+        raise ValueError(f"Unsupported metrics mode: {mode}")
+
+
+@dataclass(frozen=True)
+class StatisticsCollector:
+    field_id: int
+    iceberg_type: PrimitiveType
+    mode: MetricsMode
+    column_name: str
+
+
+class PyArrowStatisticsCollector(PreOrderSchemaVisitor[List[StatisticsCollector]]):
+    _field_id: int = 0
+    _schema: Schema
+    _properties: Dict[str, str]
+
+    def __init__(self, schema: Schema, properties: Dict[str, str]):
+        self._schema = schema
+        self._properties = properties
+
+    def schema(self, schema: Schema, struct_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        return struct_result()
+
+    def struct(
+        self, struct: StructType, field_results: List[Callable[[], List[StatisticsCollector]]]
+    ) -> List[StatisticsCollector]:
+        return list(chain(*[result() for result in field_results]))
+
+    def field(self, field: NestedField, field_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = field.field_id
+        result = field_result()
+        return result
+
+    def list(self, list_type: ListType, element_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = list_type.element_id
+        return element_result()
+
+    def map(
+        self,
+        map_type: MapType,
+        key_result: Callable[[], List[StatisticsCollector]],
+        value_result: Callable[[], List[StatisticsCollector]],
+    ) -> List[StatisticsCollector]:
+        self._field_id = map_type.key_id
+        k = key_result()
+        self._field_id = map_type.value_id
+        v = value_result()
+        return k + v
+
+    def primitive(self, primitive: PrimitiveType) -> List[StatisticsCollector]:
+        column_name = self._schema.find_column_name(self._field_id)
+        if column_name is None:
+            raise ValueError(f"Column for field {self._field_id} not found")
+
+        metrics_mode = MetricsMode(MetricModeTypes.TRUNCATE, DEFAULT_TRUNCATION_LENGTH)
+
+        default_mode = self._properties.get(DEFAULT_METRICS_MODE_KEY)
+        if default_mode:
+            metrics_mode = match_metrics_mode(default_mode)
+
+        col_mode = self._properties.get(f"{COLUMN_METRICS_MODE_KEY_PREFIX}.{column_name}")
+        if col_mode:
+            metrics_mode = match_metrics_mode(col_mode)
+
+        if (
+            not (isinstance(primitive, StringType) or isinstance(primitive, BinaryType))
+            and metrics_mode.type == MetricModeTypes.TRUNCATE
+        ):
+            metrics_mode = MetricsMode(MetricModeTypes.FULL)
+
+        return [StatisticsCollector(field_id=self._field_id, iceberg_type=primitive, mode=metrics_mode, column_name=column_name)]
+
+
+def fill_parquet_file_metadata(
+    df: DataFile,
+    parquet_metadata: pq.FileMetaData,
+    file_size: int,
+    table_metadata: TableMetadata,
+) -> None:
+    """
+    Computes and fills the following fields of the DataFile object.
+
+    - file_format
+    - record_count
+    - file_size_in_bytes
+    - column_sizes
+    - value_counts
+    - null_value_counts
+    - nan_value_counts
+    - lower_bounds
+    - upper_bounds
+    - split_offsets
+
+    Args:
+        df (DataFile): A DataFile object representing the Parquet file for which metadata is to be filled.
+        parquet_metadata (pyarrow.parquet.FileMetaData): A pyarrow metadata object.
+        file_size (int): The total compressed file size cannot be retrieved from the metadata and hence has to
+            be passed here. Depending on the kind of file system and pyarrow library call used, different
+            ways to obtain this value might be appropriate.
+        table_metadata (pyiceberg.table.metadata.TableMetadata): The Iceberg table metadata. It is required to
+            compute the mapping if column position to iceberg schema type id. It's also used to set the mode
+            for column metrics collection
+    """
+    schema = next(filter(lambda s: s.schema_id == table_metadata.current_schema_id, table_metadata.schemas))
+
+    stats_columns = pre_order_visit(schema, PyArrowStatisticsCollector(schema, table_metadata.properties))
+
+    if parquet_metadata.num_columns != len(stats_columns):

Review Comment:
   I think we should leave this one in, for now, I think they are always the same, but when this is not the case, then we should be notified



##########
python/tests/io/test_pyarrow.py:
##########
@@ -16,16 +16,39 @@
 # under the License.
 # pylint: disable=protected-access,unused-argument,redefined-outer-name
 
+import math
 import os
 import tempfile
-from typing import Any, List, Optional
+import uuid
+from datetime import (

Review Comment:
   We use this for PyArrow which has the datetime classes in their public API.



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1039,335 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.values()).union({"INT96"})
+
+
+class StatsAggregator:
+    current_min: Any
+    current_max: Any
+    trunc_length: Optional[int]
+
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min = None
+        self.current_max = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        expected_physical_type = _PRIMITIVE_TO_PHYSICAL[iceberg_type]
+        if expected_physical_type != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {expected_physical_type}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        self.current_min = val if self.current_min is None else min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        self.current_max = val if self.current_max is None else max(val, self.current_max)
+
+    def get_min(self) -> bytes:
+        return self.serialize(
+            self.current_min
+            if self.trunc_length is None
+            else TruncateTransform(width=self.trunc_length).transform(self.primitive_type)(self.current_min)
+        )
+
+    def get_max(self) -> Optional[bytes]:
+        if self.current_max is None:
+            return None
+
+        if self.primitive_type == StringType():
+            if type(self.current_max) != str:
+                raise ValueError("Expected the current_max to be a string")
+
+            s_result = self.current_max[: self.trunc_length]
+            if s_result != self.current_max:
+                chars = [*s_result]
+
+                for i in range(-1, -len(s_result) - 1, -1):
+                    try:
+                        to_inc = ord(chars[i])
+                        # will raise exception if the highest unicode code is reached
+                        _next = chr(to_inc + 1)
+                        chars[i] = _next
+                        return self.serialize("".join(chars))
+                    except ValueError:
+                        pass
+                return None  # didn't find a valid upper bound
+            return self.serialize(s_result)
+        elif self.primitive_type == BinaryType():
+            if type(self.current_max) != bytes:
+                raise ValueError("Expected the current_max to be bytes")
+            b_result = self.current_max[: self.trunc_length]
+            if b_result != self.current_max:
+                _bytes = [*b_result]
+                for i in range(-1, -len(b_result) - 1, -1):
+                    if _bytes[i] < 255:
+                        _bytes[i] += 1
+                        return b"".join([i.to_bytes(1, byteorder="little") for i in _bytes])
+                return None
+
+            return self.serialize(b_result)
+        else:
+            return self.serialize(self.current_max)[: self.trunc_length]
+
+
+DEFAULT_TRUNCATION_LENGTH = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+
+class MetricModeTypes(Enum):
+    TRUNCATE = "truncate"
+    NONE = "none"
+    COUNTS = "counts"
+    FULL = "full"
+
+
+DEFAULT_METRICS_MODE_KEY = "write.metadata.metrics.default"
+COLUMN_METRICS_MODE_KEY_PREFIX = "write.metadata.metrics.column"
+
+
+@dataclass(frozen=True)
+class MetricsMode(Singleton):
+    type: MetricModeTypes
+    length: Optional[int] = None
+
+
+def match_metrics_mode(mode: str) -> MetricsMode:
+    sanitized_mode = mode.lower()
+    if sanitized_mode.startswith("truncate"):
+        m = re.match(TRUNCATION_EXPR, mode, re.IGNORECASE)
+        if m:
+            length = int(m[1])
+            if length < 1:
+                raise ValueError("Truncation length must be larger than 0")
+            return MetricsMode(MetricModeTypes.TRUNCATE, int(m[1]))
+        else:
+            raise ValueError(f"Malformed truncate: {mode}")
+    elif sanitized_mode.startswith("none"):
+        return MetricsMode(MetricModeTypes.NONE)
+    elif sanitized_mode.startswith("counts"):
+        return MetricsMode(MetricModeTypes.COUNTS)
+    elif sanitized_mode.startswith("full"):
+        return MetricsMode(MetricModeTypes.FULL)
+    else:
+        raise ValueError(f"Unsupported metrics mode: {mode}")
+
+
+@dataclass(frozen=True)
+class StatisticsCollector:
+    field_id: int
+    iceberg_type: PrimitiveType
+    mode: MetricsMode
+    column_name: str
+
+
+class PyArrowStatisticsCollector(PreOrderSchemaVisitor[List[StatisticsCollector]]):
+    _field_id: int = 0
+    _schema: Schema
+    _properties: Dict[str, str]
+
+    def __init__(self, schema: Schema, properties: Dict[str, str]):
+        self._schema = schema
+        self._properties = properties
+
+    def schema(self, schema: Schema, struct_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        return struct_result()
+
+    def struct(
+        self, struct: StructType, field_results: List[Callable[[], List[StatisticsCollector]]]
+    ) -> List[StatisticsCollector]:
+        return list(chain(*[result() for result in field_results]))
+
+    def field(self, field: NestedField, field_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = field.field_id
+        result = field_result()
+        return result
+
+    def list(self, list_type: ListType, element_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = list_type.element_id
+        return element_result()
+
+    def map(
+        self,
+        map_type: MapType,
+        key_result: Callable[[], List[StatisticsCollector]],
+        value_result: Callable[[], List[StatisticsCollector]],
+    ) -> List[StatisticsCollector]:
+        self._field_id = map_type.key_id
+        k = key_result()
+        self._field_id = map_type.value_id
+        v = value_result()
+        return k + v
+
+    def primitive(self, primitive: PrimitiveType) -> List[StatisticsCollector]:
+        column_name = self._schema.find_column_name(self._field_id)
+        if column_name is None:
+            raise ValueError(f"Column for field {self._field_id} not found")
+
+        metrics_mode = MetricsMode(MetricModeTypes.TRUNCATE, DEFAULT_TRUNCATION_LENGTH)
+
+        default_mode = self._properties.get(DEFAULT_METRICS_MODE_KEY)
+        if default_mode:
+            metrics_mode = match_metrics_mode(default_mode)
+
+        col_mode = self._properties.get(f"{COLUMN_METRICS_MODE_KEY_PREFIX}.{column_name}")
+        if col_mode:
+            metrics_mode = match_metrics_mode(col_mode)
+
+        if (
+            not (isinstance(primitive, StringType) or isinstance(primitive, BinaryType))
+            and metrics_mode.type == MetricModeTypes.TRUNCATE
+        ):
+            metrics_mode = MetricsMode(MetricModeTypes.FULL)
+
+        return [StatisticsCollector(field_id=self._field_id, iceberg_type=primitive, mode=metrics_mode, column_name=column_name)]
+
+
+def fill_parquet_file_metadata(
+    df: DataFile,
+    parquet_metadata: pq.FileMetaData,
+    file_size: int,
+    table_metadata: TableMetadata,

Review Comment:
   We could also just pass in the `stats_columns`:
   ```suggestion
       stats_columns: List[StatisticsCollector]
   ```



##########
python/tests/io/test_pyarrow.py:
##########
@@ -1345,3 +1374,655 @@ def test_pyarrow_wrap_fsspec(example_task: FileScanTask, table_schema_simple: Sc
 bar: [[1,2,3]]
 baz: [[true,false,null]]"""
     )
+
+
+def construct_test_table() -> Tuple[Any, Any, Union[TableMetadataV1, TableMetadataV2]]:
+    table_metadata = {
+        "format-version": 2,
+        "location": "s3://bucket/test/location",
+        "last-column-id": 7,
+        "current-schema-id": 0,
+        "schemas": [
+            {
+                "type": "struct",
+                "schema-id": 0,
+                "fields": [
+                    {"id": 1, "name": "strings", "required": False, "type": "string"},
+                    {"id": 2, "name": "floats", "required": False, "type": "float"},
+                    {
+                        "id": 3,
+                        "name": "list",
+                        "required": False,
+                        "type": {"type": "list", "element-id": 5, "element": "long", "element-required": False},
+                    },
+                    {
+                        "id": 4,
+                        "name": "maps",
+                        "required": False,
+                        "type": {
+                            "type": "map",
+                            "key-id": 6,
+                            "key": "long",
+                            "value-id": 7,
+                            "value": "long",
+                            "value-required": False,
+                        },
+                    },
+                ],
+            },
+        ],
+        "default-spec-id": 0,
+        "partition-specs": [{"spec-id": 0, "fields": []}],
+        "properties": {},
+    }
+
+    table_metadata = TableMetadataUtil.parse_obj(table_metadata)
+    arrow_schema = schema_to_pyarrow(table_metadata.schemas[0])
+
+    _strings = ["zzzzzzzzzzzzzzzzzzzz", "rrrrrrrrrrrrrrrrrrrr", None, "aaaaaaaaaaaaaaaaaaaa"]
+
+    _floats = [3.14, math.nan, 1.69, 100]
+
+    _list = [[1, 2, 3], [4, 5, 6], None, [7, 8, 9]]
+
+    _maps: List[Optional[Dict[int, int]]] = [
+        {1: 2, 3: 4},
+        None,
+        {5: 6},
+        {},
+    ]
+
+    table = pa.Table.from_pydict(
+        {
+            "strings": _strings,
+            "floats": _floats,
+            "list": _list,
+            "maps": _maps,
+        },
+        schema=arrow_schema,
+    )
+    metadata_collector: List[Any] = []
+
+    with pa.BufferOutputStream() as f:
+        with pq.ParquetWriter(f, table.schema, metadata_collector=metadata_collector) as writer:
+            writer.write_table(table)
+
+        return f.getvalue(), metadata_collector[0], table_metadata
+
+
+def test_record_count() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.record_count == 4
+
+
+def test_file_size() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.file_size_in_bytes == len(file_bytes)
+
+
+def test_value_counts() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.value_counts) == 5
+    assert datafile.value_counts[1] == 4
+    assert datafile.value_counts[2] == 4
+    assert datafile.value_counts[5] == 10  # 3 lists with 3 items and a None value
+    assert datafile.value_counts[6] == 5

Review Comment:
   With Spark:
   
   ```sql
   CREATE TABLE nyc.test_map_maps2 AS 
   SELECT map_from_arrays(array(1.0, 3.0), array('2', '4')) as map, array('a', 'b', 'c') as arr
   ```
   
   Schema:
   ```json
   {
   	"type": "struct",
   	"schema-id": 0,
   	"fields": [{
   		"id": 1,
   		"name": "map",
   		"required": false,
   		"type": {
   			"type": "map",
   			"key-id": 3,
   			"key": "decimal(2, 1)",
   			"value-id": 4,
   			"value": "string",
   			"value-required": false
   		}
   	}, {
   		"id": 2,
   		"name": "arr",
   		"required": false,
   		"type": {
   			"type": "list",
   			"element-id": 5,
   			"element": "string",
   			"element-required": false
   		}
   	}]
   }
   ```
   
   We don't get any stats in Spark:
   ```json
   {
   	"status": 1,
   	"snapshot_id": {
   		"long": 4895801649705337905
   	},
   	"data_file": {
   		"file_path": "s3://warehouse/nyc/test_map_maps/data/00000-95-750d8f3e-8d49-44ec-b37e-9e101e003a5d-00001.parquet",
   		"file_format": "PARQUET",
   		"partition": {},
   		"record_count": 1,
   		"file_size_in_bytes": 1438,
   		"block_size_in_bytes": 67108864,
   		"column_sizes": {
   			"array": [{
   				"key": 3,
   				"value": 57
   			}, {
   				"key": 4,
   				"value": 58
   			}, {
   				"key": 5,
   				"value": 61
   			}]
   		},
   		"value_counts": {
   			"array": [{
   				"key": 3,
   				"value": 2
   			}, {
   				"key": 4,
   				"value": 2
   			}, {
   				"key": 5,
   				"value": 3
   			}]
   		},
   		"null_value_counts": {
   			"array": [{
   				"key": 3,
   				"value": 0
   			}, {
   				"key": 4,
   				"value": 0
   			}, {
   				"key": 5,
   				"value": 0
   			}]
   		},
   		"nan_value_counts": {
   			"array": []
   		},
   		"lower_bounds": {
   			"array": []
   		},
   		"upper_bounds": {
   			"array": []
   		},
   		"key_metadata": null,
   		"split_offsets": {
   			"array": [4]
   		},
   		"sort_order_id": {
   			"int": 0
   		}
   	}
   }
   ```
   
   The stats are only computed for the primitive types (see `PyArrowStatisticsCollector`).
   



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] maxdebayser commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "maxdebayser (via GitHub)" <gi...@apache.org>.
maxdebayser commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1286039886


##########
python/tests/io/test_pyarrow.py:
##########
@@ -1345,3 +1374,655 @@ def test_pyarrow_wrap_fsspec(example_task: FileScanTask, table_schema_simple: Sc
 bar: [[1,2,3]]
 baz: [[true,false,null]]"""
     )
+
+
+def construct_test_table() -> Tuple[Any, Any, Union[TableMetadataV1, TableMetadataV2]]:
+    table_metadata = {
+        "format-version": 2,
+        "location": "s3://bucket/test/location",
+        "last-column-id": 7,
+        "current-schema-id": 0,
+        "schemas": [
+            {
+                "type": "struct",
+                "schema-id": 0,
+                "fields": [
+                    {"id": 1, "name": "strings", "required": False, "type": "string"},
+                    {"id": 2, "name": "floats", "required": False, "type": "float"},
+                    {
+                        "id": 3,
+                        "name": "list",
+                        "required": False,
+                        "type": {"type": "list", "element-id": 5, "element": "long", "element-required": False},
+                    },
+                    {
+                        "id": 4,
+                        "name": "maps",
+                        "required": False,
+                        "type": {
+                            "type": "map",
+                            "key-id": 6,
+                            "key": "long",
+                            "value-id": 7,
+                            "value": "long",
+                            "value-required": False,
+                        },
+                    },
+                ],
+            },
+        ],
+        "default-spec-id": 0,
+        "partition-specs": [{"spec-id": 0, "fields": []}],
+        "properties": {},
+    }
+
+    table_metadata = TableMetadataUtil.parse_obj(table_metadata)
+    arrow_schema = schema_to_pyarrow(table_metadata.schemas[0])
+
+    _strings = ["zzzzzzzzzzzzzzzzzzzz", "rrrrrrrrrrrrrrrrrrrr", None, "aaaaaaaaaaaaaaaaaaaa"]
+
+    _floats = [3.14, math.nan, 1.69, 100]
+
+    _list = [[1, 2, 3], [4, 5, 6], None, [7, 8, 9]]
+
+    _maps: List[Optional[Dict[int, int]]] = [
+        {1: 2, 3: 4},
+        None,
+        {5: 6},
+        {},
+    ]
+
+    table = pa.Table.from_pydict(
+        {
+            "strings": _strings,
+            "floats": _floats,
+            "list": _list,
+            "maps": _maps,
+        },
+        schema=arrow_schema,
+    )
+    metadata_collector: List[Any] = []
+
+    with pa.BufferOutputStream() as f:
+        with pq.ParquetWriter(f, table.schema, metadata_collector=metadata_collector) as writer:
+            writer.write_table(table)
+
+        return f.getvalue(), metadata_collector[0], table_metadata
+
+
+def test_record_count() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.record_count == 4
+
+
+def test_file_size() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.file_size_in_bytes == len(file_bytes)
+
+
+def test_value_counts() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.value_counts) == 5
+    assert datafile.value_counts[1] == 4
+    assert datafile.value_counts[2] == 4
+    assert datafile.value_counts[5] == 10  # 3 lists with 3 items and a None value
+    assert datafile.value_counts[6] == 5
+    assert datafile.value_counts[7] == 5
+
+
+def test_column_sizes() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.column_sizes) == 5
+    # these values are an artifact of how the write_table encodes the columns
+    assert datafile.column_sizes[1] == 116
+    assert datafile.column_sizes[2] == 89
+    assert datafile.column_sizes[5] == 151
+    assert datafile.column_sizes[6] == 117
+    assert datafile.column_sizes[7] == 117
+
+
+def test_null_and_nan_counts() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.null_value_counts) == 5
+    assert datafile.null_value_counts[1] == 1
+    assert datafile.null_value_counts[2] == 0
+    assert datafile.null_value_counts[5] == 1
+    assert datafile.null_value_counts[6] == 2
+    assert datafile.null_value_counts[7] == 2
+
+    # #arrow does not include this in the statistics
+    # assert len(datafile.nan_value_counts)  == 3
+    # assert datafile.nan_value_counts[1]    == 0
+    # assert datafile.nan_value_counts[2]    == 1
+    # assert datafile.nan_value_counts[3]    == 0
+
+
+def test_bounds() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.lower_bounds) == 5
+    assert datafile.lower_bounds[1].decode() == "aaaaaaaaaaaaaaaa"
+    assert datafile.lower_bounds[2] == STRUCT_FLOAT.pack(1.69)
+    assert datafile.lower_bounds[5] == STRUCT_INT64.pack(1)
+    assert datafile.lower_bounds[6] == STRUCT_INT64.pack(1)
+    assert datafile.lower_bounds[7] == STRUCT_INT64.pack(2)
+
+    assert len(datafile.upper_bounds) == 5
+    assert datafile.upper_bounds[1].decode() == "zzzzzzzzzzzzzzz{"
+    assert datafile.upper_bounds[2] == STRUCT_FLOAT.pack(100)
+    assert datafile.upper_bounds[5] == STRUCT_INT64.pack(9)
+    assert datafile.upper_bounds[6] == STRUCT_INT64.pack(5)
+    assert datafile.upper_bounds[7] == STRUCT_INT64.pack(6)
+
+
+def test_metrics_mode_parsing() -> None:
+    assert match_metrics_mode("none") == MetricsMode(MetricModeTypes.NONE)
+    assert match_metrics_mode("nOnE") == MetricsMode(MetricModeTypes.NONE)
+    assert match_metrics_mode("counts") == MetricsMode(MetricModeTypes.COUNTS)
+    assert match_metrics_mode("Counts") == MetricsMode(MetricModeTypes.COUNTS)
+    assert match_metrics_mode("full") == MetricsMode(MetricModeTypes.FULL)
+    assert match_metrics_mode("FuLl") == MetricsMode(MetricModeTypes.FULL)
+
+    with pytest.raises(ValueError) as exc_info:
+        match_metrics_mode(" Full")
+    assert "Unsupported metrics mode:  Full" in str(exc_info.value)
+
+    assert match_metrics_mode("truncate(16)") == MetricsMode(MetricModeTypes.TRUNCATE, 16)
+    assert match_metrics_mode("trUncatE(16)") == MetricsMode(MetricModeTypes.TRUNCATE, 16)
+    assert match_metrics_mode("trUncatE(7)") == MetricsMode(MetricModeTypes.TRUNCATE, 7)
+    assert match_metrics_mode("trUncatE(07)") == MetricsMode(MetricModeTypes.TRUNCATE, 7)
+
+    with pytest.raises(ValueError) as exc_info:
+        match_metrics_mode("trUncatE(-7)")
+    assert "Malformed truncate: trUncatE(-7)" in str(exc_info.value)
+
+    with pytest.raises(ValueError) as exc_info:
+        match_metrics_mode("trUncatE(0)")
+    assert "Truncation length must be larger than 0" in str(exc_info.value)
+
+
+def test_metrics_mode_none() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    table_metadata.properties["write.metadata.metrics.default"] = "none"
+    fill_parquet_file_metadata(
+        datafile,
+        metadata,
+        len(file_bytes),
+        table_metadata,
+    )
+
+    assert len(datafile.value_counts) == 0
+    assert len(datafile.null_value_counts) == 0
+    assert len(datafile.nan_value_counts) == 0
+    assert len(datafile.lower_bounds) == 0
+    assert len(datafile.upper_bounds) == 0
+
+
+def test_metrics_mode_counts() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    table_metadata.properties["write.metadata.metrics.default"] = "counts"
+    fill_parquet_file_metadata(
+        datafile,
+        metadata,
+        len(file_bytes),
+        table_metadata,
+    )
+
+    assert len(datafile.value_counts) == 5
+    assert len(datafile.null_value_counts) == 5
+    assert len(datafile.nan_value_counts) == 0
+    assert len(datafile.lower_bounds) == 0
+    assert len(datafile.upper_bounds) == 0
+
+
+def test_metrics_mode_full() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    table_metadata.properties["write.metadata.metrics.default"] = "full"
+    fill_parquet_file_metadata(
+        datafile,
+        metadata,
+        len(file_bytes),
+        table_metadata,
+    )
+
+    assert len(datafile.value_counts) == 5
+    assert len(datafile.null_value_counts) == 5
+    assert len(datafile.nan_value_counts) == 0
+
+    assert len(datafile.lower_bounds) == 5
+    assert datafile.lower_bounds[1].decode() == "aaaaaaaaaaaaaaaaaaaa"
+    assert datafile.lower_bounds[2] == STRUCT_FLOAT.pack(1.69)
+    assert datafile.lower_bounds[5] == STRUCT_INT64.pack(1)
+    assert datafile.lower_bounds[6] == STRUCT_INT64.pack(1)
+    assert datafile.lower_bounds[7] == STRUCT_INT64.pack(2)
+
+    assert len(datafile.upper_bounds) == 5
+    assert datafile.upper_bounds[1].decode() == "zzzzzzzzzzzzzzzzzzzz"
+    assert datafile.upper_bounds[2] == STRUCT_FLOAT.pack(100)
+    assert datafile.upper_bounds[5] == STRUCT_INT64.pack(9)
+    assert datafile.upper_bounds[6] == STRUCT_INT64.pack(5)
+    assert datafile.upper_bounds[7] == STRUCT_INT64.pack(6)
+
+
+def test_metrics_mode_non_default_trunc() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    table_metadata.properties["write.metadata.metrics.default"] = "truncate(2)"
+    fill_parquet_file_metadata(
+        datafile,
+        metadata,
+        len(file_bytes),
+        table_metadata,
+    )
+
+    assert len(datafile.value_counts) == 5
+    assert len(datafile.null_value_counts) == 5
+    assert len(datafile.nan_value_counts) == 0
+
+    assert len(datafile.lower_bounds) == 5
+    assert datafile.lower_bounds[1].decode() == "aa"
+    assert datafile.lower_bounds[2] == STRUCT_FLOAT.pack(1.69)
+    assert datafile.lower_bounds[5] == STRUCT_INT64.pack(1)
+    assert datafile.lower_bounds[6] == STRUCT_INT64.pack(1)
+    assert datafile.lower_bounds[7] == STRUCT_INT64.pack(2)
+
+    assert len(datafile.upper_bounds) == 5
+    assert datafile.upper_bounds[1].decode() == "z{"
+    assert datafile.upper_bounds[2] == STRUCT_FLOAT.pack(100)
+    assert datafile.upper_bounds[5] == STRUCT_INT64.pack(9)
+    assert datafile.upper_bounds[6] == STRUCT_INT64.pack(5)
+    assert datafile.upper_bounds[7] == STRUCT_INT64.pack(6)
+
+
+def test_column_metrics_mode() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    table_metadata.properties["write.metadata.metrics.default"] = "truncate(2)"
+    table_metadata.properties["write.metadata.metrics.column.strings"] = "none"
+    table_metadata.properties["write.metadata.metrics.column.list.element"] = "counts"
+    fill_parquet_file_metadata(
+        datafile,
+        metadata,
+        len(file_bytes),
+        table_metadata,
+    )
+
+    assert len(datafile.value_counts) == 4
+    assert len(datafile.null_value_counts) == 4
+    assert len(datafile.nan_value_counts) == 0
+
+    assert len(datafile.lower_bounds) == 3
+    assert datafile.lower_bounds[2] == STRUCT_FLOAT.pack(1.69)
+    assert 5 not in datafile.lower_bounds

Review Comment:
   Wait, there's no integer being truncated here.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1285262616


##########
python/tests/io/test_pyarrow.py:
##########
@@ -1345,3 +1374,655 @@ def test_pyarrow_wrap_fsspec(example_task: FileScanTask, table_schema_simple: Sc
 bar: [[1,2,3]]
 baz: [[true,false,null]]"""
     )
+
+
+def construct_test_table() -> Tuple[Any, Any, Union[TableMetadataV1, TableMetadataV2]]:
+    table_metadata = {
+        "format-version": 2,
+        "location": "s3://bucket/test/location",
+        "last-column-id": 7,
+        "current-schema-id": 0,
+        "schemas": [
+            {
+                "type": "struct",
+                "schema-id": 0,
+                "fields": [
+                    {"id": 1, "name": "strings", "required": False, "type": "string"},
+                    {"id": 2, "name": "floats", "required": False, "type": "float"},
+                    {
+                        "id": 3,
+                        "name": "list",
+                        "required": False,
+                        "type": {"type": "list", "element-id": 5, "element": "long", "element-required": False},
+                    },
+                    {
+                        "id": 4,
+                        "name": "maps",
+                        "required": False,
+                        "type": {
+                            "type": "map",
+                            "key-id": 6,
+                            "key": "long",
+                            "value-id": 7,
+                            "value": "long",
+                            "value-required": False,
+                        },
+                    },
+                ],
+            },
+        ],
+        "default-spec-id": 0,
+        "partition-specs": [{"spec-id": 0, "fields": []}],
+        "properties": {},
+    }
+
+    table_metadata = TableMetadataUtil.parse_obj(table_metadata)
+    arrow_schema = schema_to_pyarrow(table_metadata.schemas[0])
+
+    _strings = ["zzzzzzzzzzzzzzzzzzzz", "rrrrrrrrrrrrrrrrrrrr", None, "aaaaaaaaaaaaaaaaaaaa"]
+
+    _floats = [3.14, math.nan, 1.69, 100]
+
+    _list = [[1, 2, 3], [4, 5, 6], None, [7, 8, 9]]
+
+    _maps: List[Optional[Dict[int, int]]] = [
+        {1: 2, 3: 4},
+        None,
+        {5: 6},
+        {},
+    ]
+
+    table = pa.Table.from_pydict(
+        {
+            "strings": _strings,
+            "floats": _floats,
+            "list": _list,
+            "maps": _maps,
+        },
+        schema=arrow_schema,
+    )
+    metadata_collector: List[Any] = []
+
+    with pa.BufferOutputStream() as f:
+        with pq.ParquetWriter(f, table.schema, metadata_collector=metadata_collector) as writer:
+            writer.write_table(table)
+
+        return f.getvalue(), metadata_collector[0], table_metadata
+
+
+def test_record_count() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.record_count == 4
+
+
+def test_file_size() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.file_size_in_bytes == len(file_bytes)
+
+
+def test_value_counts() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.value_counts) == 5
+    assert datafile.value_counts[1] == 4
+    assert datafile.value_counts[2] == 4
+    assert datafile.value_counts[5] == 10  # 3 lists with 3 items and a None value
+    assert datafile.value_counts[6] == 5
+    assert datafile.value_counts[7] == 5
+
+
+def test_column_sizes() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.column_sizes) == 5
+    # these values are an artifact of how the write_table encodes the columns
+    assert datafile.column_sizes[1] == 116
+    assert datafile.column_sizes[2] == 89
+    assert datafile.column_sizes[5] == 151
+    assert datafile.column_sizes[6] == 117
+    assert datafile.column_sizes[7] == 117
+
+
+def test_null_and_nan_counts() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.null_value_counts) == 5
+    assert datafile.null_value_counts[1] == 1
+    assert datafile.null_value_counts[2] == 0
+    assert datafile.null_value_counts[5] == 1

Review Comment:
   I also think these values are incorrect. The `null` value is not in a leaf, so it should not be reported for id=5.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1281049015


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1040,291 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHISICAL_TYPES = ["BOOLEAN", "INT32", "INT64", "INT96", "FLOAT", "DOUBLE", "BYTE_ARRAY", "FIXED_LEN_BYTE_ARRAY"]
+
+
+class StatsAggregator:
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.trunc_length = trunc_length
+
+        assert physical_type_string in _PHISICAL_TYPES, f"Unknown physical type {physical_type_string}"
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+        assert (
+            _PRIMITIVE_TO_PHYSICAL[iceberg_type] == physical_type_string
+        ), f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {_PRIMITIVE_TO_PHYSICAL[iceberg_type]}"
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if type(value) == date:

Review Comment:
   Yes, that was my intention. Let's try to get this in ASAP



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1285259470


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1039,335 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.values()).union({"INT96"})
+
+
+class StatsAggregator:
+    current_min: Any
+    current_max: Any
+    trunc_length: Optional[int]
+
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min = None
+        self.current_max = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        expected_physical_type = _PRIMITIVE_TO_PHYSICAL[iceberg_type]
+        if expected_physical_type != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {expected_physical_type}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        self.current_min = val if self.current_min is None else min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        self.current_max = val if self.current_max is None else max(val, self.current_max)
+
+    def get_min(self) -> bytes:
+        return self.serialize(
+            self.current_min
+            if self.trunc_length is None
+            else TruncateTransform(width=self.trunc_length).transform(self.primitive_type)(self.current_min)
+        )
+
+    def get_max(self) -> Optional[bytes]:
+        if self.current_max is None:
+            return None
+
+        if self.primitive_type == StringType():
+            if type(self.current_max) != str:
+                raise ValueError("Expected the current_max to be a string")
+
+            s_result = self.current_max[: self.trunc_length]
+            if s_result != self.current_max:
+                chars = [*s_result]
+
+                for i in range(-1, -len(s_result) - 1, -1):
+                    try:
+                        to_inc = ord(chars[i])
+                        # will raise exception if the highest unicode code is reached
+                        _next = chr(to_inc + 1)
+                        chars[i] = _next
+                        return self.serialize("".join(chars))
+                    except ValueError:
+                        pass
+                return None  # didn't find a valid upper bound
+            return self.serialize(s_result)
+        elif self.primitive_type == BinaryType():
+            if type(self.current_max) != bytes:
+                raise ValueError("Expected the current_max to be bytes")
+            b_result = self.current_max[: self.trunc_length]
+            if b_result != self.current_max:
+                _bytes = [*b_result]
+                for i in range(-1, -len(b_result) - 1, -1):
+                    if _bytes[i] < 255:
+                        _bytes[i] += 1
+                        return b"".join([i.to_bytes(1, byteorder="little") for i in _bytes])
+                return None
+
+            return self.serialize(b_result)
+        else:
+            return self.serialize(self.current_max)[: self.trunc_length]
+
+
+DEFAULT_TRUNCATION_LENGTH = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+
+class MetricModeTypes(Enum):
+    TRUNCATE = "truncate"
+    NONE = "none"
+    COUNTS = "counts"
+    FULL = "full"
+
+
+DEFAULT_METRICS_MODE_KEY = "write.metadata.metrics.default"
+COLUMN_METRICS_MODE_KEY_PREFIX = "write.metadata.metrics.column"
+
+
+@dataclass(frozen=True)
+class MetricsMode(Singleton):
+    type: MetricModeTypes
+    length: Optional[int] = None
+
+
+def match_metrics_mode(mode: str) -> MetricsMode:
+    sanitized_mode = mode.lower()
+    if sanitized_mode.startswith("truncate"):
+        m = re.match(TRUNCATION_EXPR, mode, re.IGNORECASE)
+        if m:
+            length = int(m[1])
+            if length < 1:
+                raise ValueError("Truncation length must be larger than 0")
+            return MetricsMode(MetricModeTypes.TRUNCATE, int(m[1]))
+        else:
+            raise ValueError(f"Malformed truncate: {mode}")
+    elif sanitized_mode.startswith("none"):
+        return MetricsMode(MetricModeTypes.NONE)
+    elif sanitized_mode.startswith("counts"):
+        return MetricsMode(MetricModeTypes.COUNTS)
+    elif sanitized_mode.startswith("full"):
+        return MetricsMode(MetricModeTypes.FULL)
+    else:
+        raise ValueError(f"Unsupported metrics mode: {mode}")
+
+
+@dataclass(frozen=True)
+class StatisticsCollector:
+    field_id: int
+    iceberg_type: PrimitiveType
+    mode: MetricsMode
+    column_name: str
+
+
+class PyArrowStatisticsCollector(PreOrderSchemaVisitor[List[StatisticsCollector]]):
+    _field_id: int = 0
+    _schema: Schema
+    _properties: Dict[str, str]
+
+    def __init__(self, schema: Schema, properties: Dict[str, str]):
+        self._schema = schema
+        self._properties = properties
+
+    def schema(self, schema: Schema, struct_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        return struct_result()
+
+    def struct(
+        self, struct: StructType, field_results: List[Callable[[], List[StatisticsCollector]]]
+    ) -> List[StatisticsCollector]:
+        return list(chain(*[result() for result in field_results]))
+
+    def field(self, field: NestedField, field_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = field.field_id
+        result = field_result()
+        return result
+
+    def list(self, list_type: ListType, element_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = list_type.element_id
+        return element_result()
+
+    def map(
+        self,
+        map_type: MapType,
+        key_result: Callable[[], List[StatisticsCollector]],
+        value_result: Callable[[], List[StatisticsCollector]],
+    ) -> List[StatisticsCollector]:
+        self._field_id = map_type.key_id
+        k = key_result()
+        self._field_id = map_type.value_id
+        v = value_result()
+        return k + v
+
+    def primitive(self, primitive: PrimitiveType) -> List[StatisticsCollector]:
+        column_name = self._schema.find_column_name(self._field_id)
+        if column_name is None:
+            raise ValueError(f"Column for field {self._field_id} not found")
+
+        metrics_mode = MetricsMode(MetricModeTypes.TRUNCATE, DEFAULT_TRUNCATION_LENGTH)
+
+        default_mode = self._properties.get(DEFAULT_METRICS_MODE_KEY)
+        if default_mode:
+            metrics_mode = match_metrics_mode(default_mode)
+
+        col_mode = self._properties.get(f"{COLUMN_METRICS_MODE_KEY_PREFIX}.{column_name}")
+        if col_mode:
+            metrics_mode = match_metrics_mode(col_mode)
+
+        if (
+            not (isinstance(primitive, StringType) or isinstance(primitive, BinaryType))
+            and metrics_mode.type == MetricModeTypes.TRUNCATE
+        ):
+            metrics_mode = MetricsMode(MetricModeTypes.FULL)
+
+        return [StatisticsCollector(field_id=self._field_id, iceberg_type=primitive, mode=metrics_mode, column_name=column_name)]
+
+
+def fill_parquet_file_metadata(
+    df: DataFile,
+    parquet_metadata: pq.FileMetaData,
+    file_size: int,
+    table_metadata: TableMetadata,
+) -> None:
+    """
+    Computes and fills the following fields of the DataFile object.
+
+    - file_format
+    - record_count
+    - file_size_in_bytes
+    - column_sizes
+    - value_counts
+    - null_value_counts
+    - nan_value_counts
+    - lower_bounds
+    - upper_bounds
+    - split_offsets
+
+    Args:
+        df (DataFile): A DataFile object representing the Parquet file for which metadata is to be filled.
+        parquet_metadata (pyarrow.parquet.FileMetaData): A pyarrow metadata object.
+        file_size (int): The total compressed file size cannot be retrieved from the metadata and hence has to
+            be passed here. Depending on the kind of file system and pyarrow library call used, different
+            ways to obtain this value might be appropriate.
+        table_metadata (pyiceberg.table.metadata.TableMetadata): The Iceberg table metadata. It is required to
+            compute the mapping if column position to iceberg schema type id. It's also used to set the mode
+            for column metrics collection
+    """
+    schema = next(filter(lambda s: s.schema_id == table_metadata.current_schema_id, table_metadata.schemas))

Review Comment:
   This work should not be done by the caller. The `TableMetadata` should either provide a map by `schema_id` or should have a method that returns a `Schema` by ID.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1285258038


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1039,335 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.values()).union({"INT96"})
+
+
+class StatsAggregator:
+    current_min: Any
+    current_max: Any
+    trunc_length: Optional[int]
+
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min = None
+        self.current_max = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        expected_physical_type = _PRIMITIVE_TO_PHYSICAL[iceberg_type]
+        if expected_physical_type != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {expected_physical_type}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        self.current_min = val if self.current_min is None else min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        self.current_max = val if self.current_max is None else max(val, self.current_max)
+
+    def get_min(self) -> bytes:
+        return self.serialize(
+            self.current_min
+            if self.trunc_length is None
+            else TruncateTransform(width=self.trunc_length).transform(self.primitive_type)(self.current_min)

Review Comment:
   It seems odd to create a transform here instead of in the constructor, but I guess if this is only ever called once?



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1279077604


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1039,35 +1042,49 @@ def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Arra
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
 
 
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHISICAL_TYPES = ["BOOLEAN", "INT32", "INT64", "INT96", "FLOAT", "DOUBLE", "BYTE_ARRAY", "FIXED_LEN_BYTE_ARRAY"]
+
+
 class StatsAggregator:
-    def __init__(self, type_string: str, trunc_length: Optional[int] = None) -> None:
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
         self.current_min: Any = None
         self.current_max: Any = None
         self.trunc_length = trunc_length
-        self.primitive_type: Optional[PrimitiveType] = None
-
-        if type_string == "BOOLEAN":
-            self.primitive_type = BooleanType()
-        elif type_string == "INT32":
-            self.primitive_type = IntegerType()
-        elif type_string == "INT64":
-            self.primitive_type = LongType()
-        elif type_string == "INT96":
+
+        assert physical_type_string in _PHISICAL_TYPES, f"Unknown physical type {physical_type_string}"

Review Comment:
   Can you raise a `ValueError` instead? We try to avoid using `assert` outside of `tests/`



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1039,35 +1042,49 @@ def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Arra
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
 
 
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHISICAL_TYPES = ["BOOLEAN", "INT32", "INT64", "INT96", "FLOAT", "DOUBLE", "BYTE_ARRAY", "FIXED_LEN_BYTE_ARRAY"]
+
+
 class StatsAggregator:
-    def __init__(self, type_string: str, trunc_length: Optional[int] = None) -> None:
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
         self.current_min: Any = None
         self.current_max: Any = None
         self.trunc_length = trunc_length
-        self.primitive_type: Optional[PrimitiveType] = None
-
-        if type_string == "BOOLEAN":
-            self.primitive_type = BooleanType()
-        elif type_string == "INT32":
-            self.primitive_type = IntegerType()
-        elif type_string == "INT64":
-            self.primitive_type = LongType()
-        elif type_string == "INT96":
+
+        assert physical_type_string in _PHISICAL_TYPES, f"Unknown physical type {physical_type_string}"
+        if physical_type_string == "INT96":
             raise NotImplementedError("Statistics not implemented for INT96 physical type")
-        elif type_string == "FLOAT":
-            self.primitive_type = FloatType()
-        elif type_string == "DOUBLE":
-            self.primitive_type = DoubleType()
-        elif type_string == "BYTE_ARRAY":
-            self.primitive_type = BinaryType()
-        elif type_string == "FIXED_LEN_BYTE_ARRAY":
-            self.primitive_type = BinaryType()
-        else:
-            raise AssertionError(f"Unknown physical type {type_string}")
+        assert (

Review Comment:
   Same here, can you change this into a `ValueError`?



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1040,291 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHISICAL_TYPES = ["BOOLEAN", "INT32", "INT64", "INT96", "FLOAT", "DOUBLE", "BYTE_ARRAY", "FIXED_LEN_BYTE_ARRAY"]
+
+
+class StatsAggregator:
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.trunc_length = trunc_length
+
+        assert physical_type_string in _PHISICAL_TYPES, f"Unknown physical type {physical_type_string}"
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+        assert (
+            _PRIMITIVE_TO_PHYSICAL[iceberg_type] == physical_type_string
+        ), f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {_PRIMITIVE_TO_PHYSICAL[iceberg_type]}"
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if type(value) == date:

Review Comment:
   I think this should be handled in `conversions.py`. We're inconsistent since we handle the UUID conversion, but not the date types. I've created a separate issue for this, so we can get this in: https://github.com/apache/iceberg/issues/8191



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1039,35 +1042,49 @@ def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Arra
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
 
 
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHISICAL_TYPES = ["BOOLEAN", "INT32", "INT64", "INT96", "FLOAT", "DOUBLE", "BYTE_ARRAY", "FIXED_LEN_BYTE_ARRAY"]
+
+
 class StatsAggregator:
-    def __init__(self, type_string: str, trunc_length: Optional[int] = None) -> None:
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
         self.current_min: Any = None
         self.current_max: Any = None
         self.trunc_length = trunc_length
-        self.primitive_type: Optional[PrimitiveType] = None
-
-        if type_string == "BOOLEAN":
-            self.primitive_type = BooleanType()
-        elif type_string == "INT32":
-            self.primitive_type = IntegerType()
-        elif type_string == "INT64":
-            self.primitive_type = LongType()
-        elif type_string == "INT96":
+
+        assert physical_type_string in _PHISICAL_TYPES, f"Unknown physical type {physical_type_string}"
+        if physical_type_string == "INT96":
             raise NotImplementedError("Statistics not implemented for INT96 physical type")
-        elif type_string == "FLOAT":
-            self.primitive_type = FloatType()
-        elif type_string == "DOUBLE":
-            self.primitive_type = DoubleType()
-        elif type_string == "BYTE_ARRAY":
-            self.primitive_type = BinaryType()
-        elif type_string == "FIXED_LEN_BYTE_ARRAY":
-            self.primitive_type = BinaryType()
-        else:
-            raise AssertionError(f"Unknown physical type {type_string}")
+        assert (
+            _PRIMITIVE_TO_PHYSICAL[iceberg_type] == physical_type_string
+        ), f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {_PRIMITIVE_TO_PHYSICAL[iceberg_type]}"
+
+        self.primitive_type = iceberg_type
 
     def serialize(self, value: Any) -> bytes:
-        if type(value) == str:
-            value = value.encode()
+        if type(value) == date:
+            value = date_to_days(value)
+        elif type(value) == time:
+            value = time_object_to_micros(value)
+        elif type(value) == datetime:
+            value = datetime_to_micros(value)
+
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
         assert self.primitive_type is not None  # appease mypy

Review Comment:
   This one still required? The `primitive_type` is now a `PrimitiveType`, instead of an `Optional[PrimitiveType]`



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1013,3 +1025,271 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+class StatsAggregator:
+    def __init__(self, type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.trunc_length = trunc_length
+        self.primitive_type: Optional[PrimitiveType] = None
+
+        if type_string == "BOOLEAN":
+            self.primitive_type = BooleanType()
+        elif type_string == "INT32":
+            self.primitive_type = IntegerType()
+        elif type_string == "INT64":
+            self.primitive_type = LongType()
+        elif type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+        elif type_string == "FLOAT":
+            self.primitive_type = FloatType()
+        elif type_string == "DOUBLE":
+            self.primitive_type = DoubleType()
+        elif type_string == "BYTE_ARRAY":
+            self.primitive_type = BinaryType()
+        elif type_string == "FIXED_LEN_BYTE_ARRAY":
+            self.primitive_type = BinaryType()
+        else:
+            raise AssertionError(f"Unknown physical type {type_string}")
+
+    def serialize(self, value: Any) -> bytes:
+        if type(value) == str:
+            value = value.encode()
+        assert self.primitive_type is not None  # appease mypy
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        if self.current_min is None:
+            self.current_min = val
+        else:
+            self.current_min = min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        if self.current_max is None:
+            self.current_max = val
+        else:
+            self.current_max = max(self.current_max, val)
+
+    def get_min(self) -> bytes:
+        return self.serialize(self.current_min)[: self.trunc_length]
+
+    def get_max(self) -> bytes:
+        return self.serialize(self.current_max)[: self.trunc_length]

Review Comment:
   > And should primitive types be truncated at all? I don't think it makes a lot of sense.
   
   This is important for str and bytes, since those can grow very big. For the others, it isn't super important.



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1039,35 +1042,49 @@ def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Arra
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
 
 
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHISICAL_TYPES = ["BOOLEAN", "INT32", "INT64", "INT96", "FLOAT", "DOUBLE", "BYTE_ARRAY", "FIXED_LEN_BYTE_ARRAY"]
+
+
 class StatsAggregator:
-    def __init__(self, type_string: str, trunc_length: Optional[int] = None) -> None:
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
         self.current_min: Any = None
         self.current_max: Any = None
         self.trunc_length = trunc_length
-        self.primitive_type: Optional[PrimitiveType] = None
-
-        if type_string == "BOOLEAN":
-            self.primitive_type = BooleanType()
-        elif type_string == "INT32":
-            self.primitive_type = IntegerType()
-        elif type_string == "INT64":
-            self.primitive_type = LongType()
-        elif type_string == "INT96":
+
+        assert physical_type_string in _PHISICAL_TYPES, f"Unknown physical type {physical_type_string}"
+        if physical_type_string == "INT96":

Review Comment:
   As a side-note. Technically this should be impossible because the Iceberg schema is used for writing. There is no INT96 in the Iceberg spec, so that should not happen, but no harm in leaving this check-in there.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] maxdebayser commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "maxdebayser (via GitHub)" <gi...@apache.org>.
maxdebayser commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1288926377


##########
python/tests/io/test_pyarrow.py:
##########
@@ -1345,3 +1374,655 @@ def test_pyarrow_wrap_fsspec(example_task: FileScanTask, table_schema_simple: Sc
 bar: [[1,2,3]]
 baz: [[true,false,null]]"""
     )
+
+
+def construct_test_table() -> Tuple[Any, Any, Union[TableMetadataV1, TableMetadataV2]]:
+    table_metadata = {
+        "format-version": 2,
+        "location": "s3://bucket/test/location",
+        "last-column-id": 7,
+        "current-schema-id": 0,
+        "schemas": [
+            {
+                "type": "struct",
+                "schema-id": 0,
+                "fields": [
+                    {"id": 1, "name": "strings", "required": False, "type": "string"},
+                    {"id": 2, "name": "floats", "required": False, "type": "float"},
+                    {
+                        "id": 3,
+                        "name": "list",
+                        "required": False,
+                        "type": {"type": "list", "element-id": 5, "element": "long", "element-required": False},
+                    },
+                    {
+                        "id": 4,
+                        "name": "maps",
+                        "required": False,
+                        "type": {
+                            "type": "map",
+                            "key-id": 6,
+                            "key": "long",
+                            "value-id": 7,
+                            "value": "long",
+                            "value-required": False,
+                        },
+                    },
+                ],
+            },
+        ],
+        "default-spec-id": 0,
+        "partition-specs": [{"spec-id": 0, "fields": []}],
+        "properties": {},
+    }
+
+    table_metadata = TableMetadataUtil.parse_obj(table_metadata)
+    arrow_schema = schema_to_pyarrow(table_metadata.schemas[0])
+
+    _strings = ["zzzzzzzzzzzzzzzzzzzz", "rrrrrrrrrrrrrrrrrrrr", None, "aaaaaaaaaaaaaaaaaaaa"]
+
+    _floats = [3.14, math.nan, 1.69, 100]
+
+    _list = [[1, 2, 3], [4, 5, 6], None, [7, 8, 9]]
+
+    _maps: List[Optional[Dict[int, int]]] = [
+        {1: 2, 3: 4},
+        None,
+        {5: 6},
+        {},
+    ]
+
+    table = pa.Table.from_pydict(
+        {
+            "strings": _strings,
+            "floats": _floats,
+            "list": _list,
+            "maps": _maps,
+        },
+        schema=arrow_schema,
+    )
+    metadata_collector: List[Any] = []
+
+    with pa.BufferOutputStream() as f:
+        with pq.ParquetWriter(f, table.schema, metadata_collector=metadata_collector) as writer:
+            writer.write_table(table)
+
+        return f.getvalue(), metadata_collector[0], table_metadata
+
+
+def test_record_count() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.record_count == 4
+
+
+def test_file_size() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.file_size_in_bytes == len(file_bytes)
+
+
+def test_value_counts() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.value_counts) == 5
+    assert datafile.value_counts[1] == 4
+    assert datafile.value_counts[2] == 4
+    assert datafile.value_counts[5] == 10  # 3 lists with 3 items and a None value
+    assert datafile.value_counts[6] == 5
+    assert datafile.value_counts[7] == 5
+
+
+def test_column_sizes() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.column_sizes) == 5
+    # these values are an artifact of how the write_table encodes the columns
+    assert datafile.column_sizes[1] == 116
+    assert datafile.column_sizes[2] == 89
+    assert datafile.column_sizes[5] == 151
+    assert datafile.column_sizes[6] == 117
+    assert datafile.column_sizes[7] == 117
+
+
+def test_null_and_nan_counts() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.null_value_counts) == 5
+    assert datafile.null_value_counts[1] == 1
+    assert datafile.null_value_counts[2] == 0
+    assert datafile.null_value_counts[5] == 1

Review Comment:
   This is how pyarrow is returning these values. Unless we look at the actual table it seems to me that the only options are not returning this statistic or returning it as is.
   ```
   <pyarrow._parquet.ColumnChunkMetaData object at 0x7fbd4a8a15e0>
     file_offset: 543
     file_path: 
     physical_type: INT64
     num_values: 10
     path_in_schema: list.list.element
     is_stats_set: True
     statistics:
       <pyarrow._parquet.Statistics object at 0x7fbd4a8a1130>
         has_min_max: True
         min: 1
         max: 9
         null_count: 1
         distinct_count: 0
         num_values: 9
         physical_type: INT64
         logical_type: None
         converted_type (legacy): NONE
     compression: SNAPPY
     encodings: ('RLE_DICTIONARY', 'PLAIN', 'RLE')
     has_dictionary_page: True
     dictionary_page_offset: 392
     data_page_offset: 455
     total_compressed_size: 151
     total_uncompressed_size: 174
   ```



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1267853906


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1013,3 +1025,271 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+class StatsAggregator:
+    def __init__(self, type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.trunc_length = trunc_length
+        self.primitive_type: Optional[PrimitiveType] = None
+
+        if type_string == "BOOLEAN":
+            self.primitive_type = BooleanType()
+        elif type_string == "INT32":
+            self.primitive_type = IntegerType()
+        elif type_string == "INT64":
+            self.primitive_type = LongType()
+        elif type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+        elif type_string == "FLOAT":
+            self.primitive_type = FloatType()
+        elif type_string == "DOUBLE":
+            self.primitive_type = DoubleType()
+        elif type_string == "BYTE_ARRAY":
+            self.primitive_type = BinaryType()
+        elif type_string == "FIXED_LEN_BYTE_ARRAY":
+            self.primitive_type = BinaryType()
+        else:
+            raise AssertionError(f"Unknown physical type {type_string}")

Review Comment:
   Yes, I think we can do this. I would suggest creating a visitor that will convert an Iceberg type to an Arrow string type. We have to make sure that we test for all the primitive types.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1267877617


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1013,3 +1025,271 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+class StatsAggregator:
+    def __init__(self, type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.trunc_length = trunc_length
+        self.primitive_type: Optional[PrimitiveType] = None
+
+        if type_string == "BOOLEAN":
+            self.primitive_type = BooleanType()
+        elif type_string == "INT32":
+            self.primitive_type = IntegerType()
+        elif type_string == "INT64":
+            self.primitive_type = LongType()
+        elif type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+        elif type_string == "FLOAT":
+            self.primitive_type = FloatType()
+        elif type_string == "DOUBLE":
+            self.primitive_type = DoubleType()
+        elif type_string == "BYTE_ARRAY":
+            self.primitive_type = BinaryType()
+        elif type_string == "FIXED_LEN_BYTE_ARRAY":
+            self.primitive_type = BinaryType()
+        else:
+            raise AssertionError(f"Unknown physical type {type_string}")
+
+    def serialize(self, value: Any) -> bytes:
+        if type(value) == str:
+            value = value.encode()
+        assert self.primitive_type is not None  # appease mypy
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        if self.current_min is None:
+            self.current_min = val
+        else:
+            self.current_min = min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        if self.current_max is None:
+            self.current_max = val
+        else:
+            self.current_max = max(self.current_max, val)
+
+    def get_min(self) -> bytes:
+        return self.serialize(self.current_min)[: self.trunc_length]
+
+    def get_max(self) -> bytes:
+        return self.serialize(self.current_max)[: self.trunc_length]

Review Comment:
   Strings in Python are already utf8, so we could truncate them before converting them to bytes. I don't think we need to truncate int/float/double/etc.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] maxdebayser commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "maxdebayser (via GitHub)" <gi...@apache.org>.
maxdebayser commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1267112518


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1013,3 +1025,271 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+class StatsAggregator:
+    def __init__(self, type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.trunc_length = trunc_length
+        self.primitive_type: Optional[PrimitiveType] = None
+
+        if type_string == "BOOLEAN":
+            self.primitive_type = BooleanType()
+        elif type_string == "INT32":
+            self.primitive_type = IntegerType()
+        elif type_string == "INT64":
+            self.primitive_type = LongType()
+        elif type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+        elif type_string == "FLOAT":
+            self.primitive_type = FloatType()
+        elif type_string == "DOUBLE":
+            self.primitive_type = DoubleType()
+        elif type_string == "BYTE_ARRAY":
+            self.primitive_type = BinaryType()
+        elif type_string == "FIXED_LEN_BYTE_ARRAY":
+            self.primitive_type = BinaryType()
+        else:
+            raise AssertionError(f"Unknown physical type {type_string}")
+
+    def serialize(self, value: Any) -> bytes:
+        if type(value) == str:
+            value = value.encode()
+        assert self.primitive_type is not None  # appease mypy
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        if self.current_min is None:
+            self.current_min = val
+        else:
+            self.current_min = min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        if self.current_max is None:
+            self.current_max = val
+        else:
+            self.current_max = max(self.current_max, val)
+
+    def get_min(self) -> bytes:
+        return self.serialize(self.current_min)[: self.trunc_length]
+
+    def get_max(self) -> bytes:
+        return self.serialize(self.current_max)[: self.trunc_length]

Review Comment:
   Thanks for pointing that out. I've just read the UnicodeUtil code and it seems to me that the truncation length there is interpreted as length in characters and not in bytes. Is that understanding correct?
   
   And should primitive types be truncated at all? I don't think it makes a lot of sense.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] maxdebayser commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "maxdebayser (via GitHub)" <gi...@apache.org>.
maxdebayser commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1267109859


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1013,3 +1025,271 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+class StatsAggregator:
+    def __init__(self, type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.trunc_length = trunc_length
+        self.primitive_type: Optional[PrimitiveType] = None
+
+        if type_string == "BOOLEAN":
+            self.primitive_type = BooleanType()
+        elif type_string == "INT32":
+            self.primitive_type = IntegerType()
+        elif type_string == "INT64":
+            self.primitive_type = LongType()
+        elif type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+        elif type_string == "FLOAT":
+            self.primitive_type = FloatType()
+        elif type_string == "DOUBLE":
+            self.primitive_type = DoubleType()
+        elif type_string == "BYTE_ARRAY":
+            self.primitive_type = BinaryType()
+        elif type_string == "FIXED_LEN_BYTE_ARRAY":
+            self.primitive_type = BinaryType()
+        else:
+            raise AssertionError(f"Unknown physical type {type_string}")
+
+    def serialize(self, value: Any) -> bytes:
+        if type(value) == str:
+            value = value.encode()
+        assert self.primitive_type is not None  # appease mypy
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        if self.current_min is None:
+            self.current_min = val
+        else:
+            self.current_min = min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        if self.current_max is None:
+            self.current_max = val
+        else:
+            self.current_max = max(self.current_max, val)
+
+    def get_min(self) -> bytes:
+        return self.serialize(self.current_min)[: self.trunc_length]
+
+    def get_max(self) -> bytes:
+        return self.serialize(self.current_max)[: self.trunc_length]
+
+
+DEFAULT_TRUNCATION_LENGHT = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+
+class MetricModeTypes(Enum):
+    TRUNCATE = "truncate"
+    NONE = "none"
+    COUNTS = "counts"
+    FULL = "full"
+
+
+DEFAULT_METRICS_MODE_KEY = "write.metadata.metrics.default"
+COLUMN_METRICS_MODE_KEY = "write.metadata.metrics.column"

Review Comment:
   Yes, I've changed the name to reflect that.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko merged pull request #7831: Python: Compute parquet stats

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko merged PR #7831:
URL: https://github.com/apache/iceberg/pull/7831


-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on pull request #7831: Compute parquet stats

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#issuecomment-1590003401

   It looks like the metadata is also available on the plain writer, where we don't have to write a dataset:
   
   ```python
   ➜  Desktop python3                       
   Python 3.11.3 (main, Apr  7 2023, 20:13:31) [Clang 14.0.0 (clang-1400.0.29.202)] on darwin
   Type "help", "copyright", "credits" or "license" for more information.
   >>> import pyarrow as pa
   >>> table = pa.table({'n_legs': [2, 2, 4, 4, 5, 100],
   ...                   'animal': ["Flamingo", "Parrot", "Dog", "Horse", "Brittle stars", "Centipede"]})
   >>> import pyarrow.parquet as pq
   >>> 
   >>> writer = pq.ParquetWriter('/tmp/vo.parquet', table.schema)
   >>> writer.write_table(table)
   >>> writer.close()
   >>> writer.writer.metadata.row_group(0)
   <pyarrow._parquet.RowGroupMetaData object at 0x1375b9080>
     num_columns: 2
     num_rows: 6
     total_byte_size: 256
   >>> writer.writer.metadata.row_group(0).to_dict()
   {
   	'num_columns': 2,
   	'num_rows': 6,
   	'total_byte_size': 256,
   	'columns': [{
   		'file_offset': 119,
   		'file_path': '',
   		'physical_type': 'INT64',
   		'num_values': 6,
   		'path_in_schema': 'n_legs',
   		'is_stats_set': True,
   		'statistics': {
   			'has_min_max': True,
   			'min': 2,
   			'max': 100,
   			'null_count': 0,
   			'distinct_count': 0,
   			'num_values': 6,
   			'physical_type': 'INT64'
   		},
   		'compression': 'SNAPPY',
   		'encodings': ('RLE_DICTIONARY', 'PLAIN', 'RLE'),
   		'has_dictionary_page': True,
   		'dictionary_page_offset': 4,
   		'data_page_offset': 46,
   		'total_compressed_size': 115,
   		'total_uncompressed_size': 117
   	}, {
   		'file_offset': 359,
   		'file_path': '',
   		'physical_type': 'BYTE_ARRAY',
   		'num_values': 6,
   		'path_in_schema': 'animal',
   		'is_stats_set': True,
   		'statistics': {
   			'has_min_max': True,
   			'min': 'Brittle stars',
   			'max': 'Parrot',
   			'null_count': 0,
   			'distinct_count': 0,
   			'num_values': 6,
   			'physical_type': 'BYTE_ARRAY'
   		},
   		'compression': 'SNAPPY',
   		'encodings': ('RLE_DICTIONARY', 'PLAIN', 'RLE'),
   		'has_dictionary_page': True,
   		'dictionary_page_offset': 215,
   		'data_page_offset': 302,
   		'total_compressed_size': 144,
   		'total_uncompressed_size': 139
   	}]
   }
   ```


-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on a diff in pull request #7831: Compute parquet stats

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1229173359


##########
python/pyiceberg/utils/file_stats.py:
##########
@@ -0,0 +1,164 @@
+#  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.
+
+from pyiceberg.manifest import DataFile, FileFormat
+import pyarrow.parquet as pq
+import pyarrow.compute as pc
+import pyarrow as pa
+import struct
+import datetime
+
+BOUND_TRUNCATED_LENGHT = 16
+
+# Serialization rules: https://iceberg.apache.org/spec/#binary-single-value-serialization
+#
+# Type      Binary serialization
+# boolean   0x00 for false, non-zero byte for true
+# int       Stored as 4-byte little-endian
+# long      Stored as 8-byte little-endian
+# float     Stored as 4-byte little-endian
+# double    Stored as 8-byte little-endian
+# date      Stores days from the 1970-01-01 in an 4-byte little-endian int
+# time      Stores microseconds from midnight in an 8-byte little-endian long
+# timestamp without zone	Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long
+# timestamp with zone	Stores microseconds from 1970-01-01 00:00:00.000000 UTC in an 8-byte little-endian long
+# string    UTF-8 bytes (without length)
+# uuid      16-byte big-endian value, see example in Appendix B
+# fixed(L)  Binary value
+# binary    Binary value (without length)
+#
+def serialize_to_binary(scalar: pa.Scalar) -> bytes:

Review Comment:
   We could reuse existing logic by first converting the PyArrow DataType to an Iceberg type using `pyarrow_to_schema(datatype)`, and then reuse `to_bytes` in `conversion.py`.



##########
python/pyiceberg/utils/file_stats.py:
##########
@@ -0,0 +1,164 @@
+#  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.
+
+from pyiceberg.manifest import DataFile, FileFormat
+import pyarrow.parquet as pq
+import pyarrow.compute as pc
+import pyarrow as pa
+import struct
+import datetime
+
+BOUND_TRUNCATED_LENGHT = 16
+
+# Serialization rules: https://iceberg.apache.org/spec/#binary-single-value-serialization
+#
+# Type      Binary serialization
+# boolean   0x00 for false, non-zero byte for true
+# int       Stored as 4-byte little-endian
+# long      Stored as 8-byte little-endian
+# float     Stored as 4-byte little-endian
+# double    Stored as 8-byte little-endian
+# date      Stores days from the 1970-01-01 in an 4-byte little-endian int
+# time      Stores microseconds from midnight in an 8-byte little-endian long
+# timestamp without zone	Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long
+# timestamp with zone	Stores microseconds from 1970-01-01 00:00:00.000000 UTC in an 8-byte little-endian long
+# string    UTF-8 bytes (without length)
+# uuid      16-byte big-endian value, see example in Appendix B
+# fixed(L)  Binary value
+# binary    Binary value (without length)
+#
+def serialize_to_binary(scalar: pa.Scalar) -> bytes:
+    value = scalar.as_py()
+    if isinstance(scalar, pa.BooleanScalar):
+        return struct.pack('?', value)

Review Comment:
   For performance reasons, it is best to re-use the `struct`'s, see `conversions.py`



##########
python/pyiceberg/utils/file_stats.py:
##########
@@ -0,0 +1,164 @@
+#  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.
+
+from pyiceberg.manifest import DataFile, FileFormat
+import pyarrow.parquet as pq
+import pyarrow.compute as pc
+import pyarrow as pa
+import struct
+import datetime
+
+BOUND_TRUNCATED_LENGHT = 16
+
+# Serialization rules: https://iceberg.apache.org/spec/#binary-single-value-serialization
+#
+# Type      Binary serialization
+# boolean   0x00 for false, non-zero byte for true
+# int       Stored as 4-byte little-endian
+# long      Stored as 8-byte little-endian
+# float     Stored as 4-byte little-endian
+# double    Stored as 8-byte little-endian
+# date      Stores days from the 1970-01-01 in an 4-byte little-endian int
+# time      Stores microseconds from midnight in an 8-byte little-endian long
+# timestamp without zone	Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long
+# timestamp with zone	Stores microseconds from 1970-01-01 00:00:00.000000 UTC in an 8-byte little-endian long
+# string    UTF-8 bytes (without length)
+# uuid      16-byte big-endian value, see example in Appendix B
+# fixed(L)  Binary value
+# binary    Binary value (without length)
+#
+def serialize_to_binary(scalar: pa.Scalar) -> bytes:
+    value = scalar.as_py()
+    if isinstance(scalar, pa.BooleanScalar):
+        return struct.pack('?', value)
+    elif isinstance(scalar, (pa.Int8Scalar, pa.UInt8Scalar)):
+        return struct.pack('<b', value)
+    elif isinstance(scalar, (pa.Int16Scalar, pa.UInt16Scalar)):
+        return struct.pack('<h', value)
+    elif isinstance(scalar, (pa.Int32Scalar, pa.UInt32Scalar)):
+        return struct.pack('<i', value)
+    elif isinstance(scalar, (pa.Int64Scalar, pa.UInt64Scalar)):
+        return struct.pack('<q', value)
+    elif isinstance(scalar, pa.FloatScalar):
+        return struct.pack('<f', value)
+    elif isinstance(scalar, pa.DoubleScalar):
+        return struct.pack('<d', value)
+    elif isinstance(scalar, pa.StringScalar):
+        return value.encode('utf-8')
+    elif isinstance(scalar, pa.BinaryScalar):
+        return value
+    elif isinstance(scalar, (pa.Date32Scalar, pa.Date64Scalar)):
+        epoch = datetime.date(1970, 1, 1)
+        days = (value - epoch).days
+        return struct.pack('<i', days)
+    elif isinstance(scalar, (pa.Time32Scalar, pa.Time64Scalar)):
+        microseconds = int(value.hour * 60 * 60 * 1e6 +
+                        value.minute * 60 * 1e6 +
+                        value.second * 1e6 +
+                        value.microsecond)
+        return struct.pack('<q', microseconds)
+    elif isinstance(scalar, pa.TimestampScalar):
+        epoch = datetime.datetime(1970, 1, 1)
+        microseconds = int((value - epoch).total_seconds() * 1e6)
+        return struct.pack('<q', microseconds)
+    else:
+        raise TypeError('Unsupported type: {}'.format(type(scalar)))
+
+
+def fill_parquet_file_metadata(df: DataFile, file_object: pa.NativeFile, table: pa.Table = None) -> None:
+    """
+    Computes and fills the following fields of the DataFile object:
+
+    - file_format
+    - record_count
+    - file_size_in_bytes
+    - column_sizes
+    - value_counts
+    - null_value_counts
+    - nan_value_counts
+    - lower_bounds
+    - upper_bounds
+    - split_offsets
+    
+    Args:
+        df (DataFile): A DataFile object representing the Parquet file for which metadata is to be filled.
+        file_object (pa.NativeFile): A pyarrow NativeFile object pointing to the location where the 
+            Parquet file is stored.
+        table (pa.Table, optional): If the metadata is computed while writing a pyarrow Table to parquet
+            the table can be passed to compute the column statistics. If absent the table will be read
+            from file_object using pyarrow.parquet.read_table.
+    """
+    
+    parquet_file = pq.ParquetFile(file_object)
+    metadata = parquet_file.metadata
+
+    column_sizes = {}
+    value_counts = {}
+
+    for r in range(metadata.num_row_groups):
+        for c in range(metadata.num_columns):
+            column_sizes[c+1] = column_sizes.get(c+1, 0) + metadata.row_group(r).column(c).total_compressed_size
+            value_counts[c+1] = value_counts.get(c+1, 0) + metadata.row_group(r).column(c).num_values
+
+
+    # References:
+    # https://github.com/apache/iceberg/blob/fc381a81a1fdb8f51a0637ca27cd30673bd7aad3/parquet/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java#L232
+    # https://github.com/apache/parquet-mr/blob/ac29db4611f86a07cc6877b416aa4b183e09b353/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/ColumnChunkMetaData.java#L184
+    split_offsets = []
+    for r in range(metadata.num_row_groups):
+        data_offset       = metadata.row_group(r).column(0).data_page_offset
+        dictionary_offset = metadata.row_group(r).column(0).dictionary_page_offset
+        if metadata.row_group(r).column(0).has_dictionary_page and dictionary_offset < data_offset:
+            split_offsets.append(dictionary_offset)
+        else:
+            split_offsets.append(data_offset)
+
+    split_offsets.sort()
+
+    if table is None:
+        table = pa.parquet.read_table(file_object)
+
+    null_value_counts = {}
+    nan_value_counts  = {}
+    lower_bounds      = {}
+    upper_bounds      = {}
+
+    for c in range(metadata.num_columns):
+        null_value_counts[c+1] = table.filter(pc.field(c).is_null(nan_is_null=False)).num_rows

Review Comment:
   This combines comment re-using `to_bytes` and fetching the field-id above.
   
   Here we also have to index the field IDs. When we get the field from the Iceberg schema, we also know the type. We could invoke `to_bytes` directly with the Iceberg type.



##########
python/pyiceberg/utils/file_stats.py:
##########
@@ -0,0 +1,164 @@
+#  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.
+
+from pyiceberg.manifest import DataFile, FileFormat
+import pyarrow.parquet as pq
+import pyarrow.compute as pc
+import pyarrow as pa
+import struct
+import datetime
+
+BOUND_TRUNCATED_LENGHT = 16
+
+# Serialization rules: https://iceberg.apache.org/spec/#binary-single-value-serialization
+#
+# Type      Binary serialization
+# boolean   0x00 for false, non-zero byte for true
+# int       Stored as 4-byte little-endian
+# long      Stored as 8-byte little-endian
+# float     Stored as 4-byte little-endian
+# double    Stored as 8-byte little-endian
+# date      Stores days from the 1970-01-01 in an 4-byte little-endian int
+# time      Stores microseconds from midnight in an 8-byte little-endian long
+# timestamp without zone	Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long
+# timestamp with zone	Stores microseconds from 1970-01-01 00:00:00.000000 UTC in an 8-byte little-endian long
+# string    UTF-8 bytes (without length)
+# uuid      16-byte big-endian value, see example in Appendix B
+# fixed(L)  Binary value
+# binary    Binary value (without length)
+#
+def serialize_to_binary(scalar: pa.Scalar) -> bytes:
+    value = scalar.as_py()
+    if isinstance(scalar, pa.BooleanScalar):
+        return struct.pack('?', value)
+    elif isinstance(scalar, (pa.Int8Scalar, pa.UInt8Scalar)):
+        return struct.pack('<b', value)
+    elif isinstance(scalar, (pa.Int16Scalar, pa.UInt16Scalar)):
+        return struct.pack('<h', value)
+    elif isinstance(scalar, (pa.Int32Scalar, pa.UInt32Scalar)):
+        return struct.pack('<i', value)
+    elif isinstance(scalar, (pa.Int64Scalar, pa.UInt64Scalar)):
+        return struct.pack('<q', value)
+    elif isinstance(scalar, pa.FloatScalar):
+        return struct.pack('<f', value)
+    elif isinstance(scalar, pa.DoubleScalar):
+        return struct.pack('<d', value)
+    elif isinstance(scalar, pa.StringScalar):
+        return value.encode('utf-8')
+    elif isinstance(scalar, pa.BinaryScalar):
+        return value
+    elif isinstance(scalar, (pa.Date32Scalar, pa.Date64Scalar)):
+        epoch = datetime.date(1970, 1, 1)
+        days = (value - epoch).days
+        return struct.pack('<i', days)
+    elif isinstance(scalar, (pa.Time32Scalar, pa.Time64Scalar)):
+        microseconds = int(value.hour * 60 * 60 * 1e6 +
+                        value.minute * 60 * 1e6 +
+                        value.second * 1e6 +
+                        value.microsecond)
+        return struct.pack('<q', microseconds)
+    elif isinstance(scalar, pa.TimestampScalar):
+        epoch = datetime.datetime(1970, 1, 1)
+        microseconds = int((value - epoch).total_seconds() * 1e6)
+        return struct.pack('<q', microseconds)
+    else:
+        raise TypeError('Unsupported type: {}'.format(type(scalar)))
+
+
+def fill_parquet_file_metadata(df: DataFile, file_object: pa.NativeFile, table: pa.Table = None) -> None:
+    """
+    Computes and fills the following fields of the DataFile object:
+
+    - file_format
+    - record_count
+    - file_size_in_bytes
+    - column_sizes
+    - value_counts
+    - null_value_counts
+    - nan_value_counts
+    - lower_bounds
+    - upper_bounds
+    - split_offsets
+    
+    Args:
+        df (DataFile): A DataFile object representing the Parquet file for which metadata is to be filled.
+        file_object (pa.NativeFile): A pyarrow NativeFile object pointing to the location where the 
+            Parquet file is stored.
+        table (pa.Table, optional): If the metadata is computed while writing a pyarrow Table to parquet
+            the table can be passed to compute the column statistics. If absent the table will be read
+            from file_object using pyarrow.parquet.read_table.
+    """
+    
+    parquet_file = pq.ParquetFile(file_object)
+    metadata = parquet_file.metadata
+
+    column_sizes = {}
+    value_counts = {}
+
+    for r in range(metadata.num_row_groups):
+        for c in range(metadata.num_columns):
+            column_sizes[c+1] = column_sizes.get(c+1, 0) + metadata.row_group(r).column(c).total_compressed_size

Review Comment:
   FieldIDs are a fundamental concept of Iceberg. Instead of relying on column names (or positions), each of the fields gets a unique field-id assigned that is monotonically increasing when new fields are being added (and are never re-used). These field IDs are used for example:
   
   - To safely rename a column (you change the name, but the ID is still pointing at the existing field).
   - To drop a column, and create a new column with the same name. Give this a try in Spark/Hive on a plain Parquet table.
   
   The `column_sizes` is also indexed by the field-id. What we need to do is:
   
   ```
   >>> metadata_collector[0].row_group(0).to_dict()
   {
   	'num_columns': 2,
   	'num_rows': 6,
   	'total_byte_size': 256,
   	'columns': [{
   		'file_offset': 119,
   		'file_path': 'c569c5eaf90c4395885f31e012068b69-0.parquet',
   		'physical_type': 'INT64',
   		'num_values': 6,
   		'path_in_schema': 'n_legs',
   		'is_stats_set': True,
   		'statistics': {
   			'has_min_max': True,
   			'min': 2,
   			'max': 100,
   			'null_count': 0,
   			'distinct_count': 0,
   			'num_values': 6,
   			'physical_type': 'INT64'
   		},
   		'compression': 'SNAPPY',
   		'encodings': ('PLAIN_DICTIONARY', 'PLAIN', 'RLE'),
   		'has_dictionary_page': True,
   		'dictionary_page_offset': 4,
   		'data_page_offset': 46,
   		'total_compressed_size': 115,
   		'total_uncompressed_size': 117
   	}
   }
   ```
   Take the `path_in_schema`, in this case, `n_legs`. Using the current table schema you can do `schema.find_field('n_legs')` and that will return the `Field` that contains the field-id `schema.field_id`.



##########
python/pyiceberg/utils/file_stats.py:
##########
@@ -0,0 +1,164 @@
+#  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.
+
+from pyiceberg.manifest import DataFile, FileFormat
+import pyarrow.parquet as pq
+import pyarrow.compute as pc
+import pyarrow as pa
+import struct
+import datetime
+
+BOUND_TRUNCATED_LENGHT = 16
+
+# Serialization rules: https://iceberg.apache.org/spec/#binary-single-value-serialization
+#
+# Type      Binary serialization
+# boolean   0x00 for false, non-zero byte for true
+# int       Stored as 4-byte little-endian
+# long      Stored as 8-byte little-endian
+# float     Stored as 4-byte little-endian
+# double    Stored as 8-byte little-endian
+# date      Stores days from the 1970-01-01 in an 4-byte little-endian int
+# time      Stores microseconds from midnight in an 8-byte little-endian long
+# timestamp without zone	Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long
+# timestamp with zone	Stores microseconds from 1970-01-01 00:00:00.000000 UTC in an 8-byte little-endian long
+# string    UTF-8 bytes (without length)
+# uuid      16-byte big-endian value, see example in Appendix B
+# fixed(L)  Binary value
+# binary    Binary value (without length)
+#
+def serialize_to_binary(scalar: pa.Scalar) -> bytes:
+    value = scalar.as_py()
+    if isinstance(scalar, pa.BooleanScalar):
+        return struct.pack('?', value)
+    elif isinstance(scalar, (pa.Int8Scalar, pa.UInt8Scalar)):
+        return struct.pack('<b', value)
+    elif isinstance(scalar, (pa.Int16Scalar, pa.UInt16Scalar)):
+        return struct.pack('<h', value)
+    elif isinstance(scalar, (pa.Int32Scalar, pa.UInt32Scalar)):
+        return struct.pack('<i', value)
+    elif isinstance(scalar, (pa.Int64Scalar, pa.UInt64Scalar)):
+        return struct.pack('<q', value)
+    elif isinstance(scalar, pa.FloatScalar):
+        return struct.pack('<f', value)
+    elif isinstance(scalar, pa.DoubleScalar):
+        return struct.pack('<d', value)
+    elif isinstance(scalar, pa.StringScalar):
+        return value.encode('utf-8')
+    elif isinstance(scalar, pa.BinaryScalar):
+        return value
+    elif isinstance(scalar, (pa.Date32Scalar, pa.Date64Scalar)):
+        epoch = datetime.date(1970, 1, 1)
+        days = (value - epoch).days
+        return struct.pack('<i', days)
+    elif isinstance(scalar, (pa.Time32Scalar, pa.Time64Scalar)):
+        microseconds = int(value.hour * 60 * 60 * 1e6 +
+                        value.minute * 60 * 1e6 +
+                        value.second * 1e6 +
+                        value.microsecond)
+        return struct.pack('<q', microseconds)
+    elif isinstance(scalar, pa.TimestampScalar):
+        epoch = datetime.datetime(1970, 1, 1)
+        microseconds = int((value - epoch).total_seconds() * 1e6)
+        return struct.pack('<q', microseconds)
+    else:
+        raise TypeError('Unsupported type: {}'.format(type(scalar)))
+
+
+def fill_parquet_file_metadata(df: DataFile, file_object: pa.NativeFile, table: pa.Table = None) -> None:
+    """
+    Computes and fills the following fields of the DataFile object:
+
+    - file_format
+    - record_count
+    - file_size_in_bytes
+    - column_sizes
+    - value_counts
+    - null_value_counts
+    - nan_value_counts
+    - lower_bounds
+    - upper_bounds
+    - split_offsets
+    
+    Args:
+        df (DataFile): A DataFile object representing the Parquet file for which metadata is to be filled.
+        file_object (pa.NativeFile): A pyarrow NativeFile object pointing to the location where the 
+            Parquet file is stored.
+        table (pa.Table, optional): If the metadata is computed while writing a pyarrow Table to parquet
+            the table can be passed to compute the column statistics. If absent the table will be read
+            from file_object using pyarrow.parquet.read_table.
+    """
+    
+    parquet_file = pq.ParquetFile(file_object)

Review Comment:
   This is my main concern, we read the file here, and I would like to see if we can collect the statistics while writing.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] maxdebayser commented on pull request #7831: Compute parquet stats

Posted by "maxdebayser (via GitHub)" <gi...@apache.org>.
maxdebayser commented on PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#issuecomment-1596144303

   > It looks like the metadata is also available on the plain writer, where we don't have to write a dataset:
   > 
   > ```python
   > ➜  Desktop python3                       
   > Python 3.11.3 (main, Apr  7 2023, 20:13:31) [Clang 14.0.0 (clang-1400.0.29.202)] on darwin
   > Type "help", "copyright", "credits" or "license" for more information.
   > >>> import pyarrow as pa
   > >>> table = pa.table({'n_legs': [2, 2, 4, 4, 5, 100],
   > ...                   'animal': ["Flamingo", "Parrot", "Dog", "Horse", "Brittle stars", "Centipede"]})
   > >>> import pyarrow.parquet as pq
   > >>> 
   > >>> writer = pq.ParquetWriter('/tmp/vo.parquet', table.schema)
   > >>> writer.write_table(table)
   > >>> writer.close()
   > >>> writer.writer.metadata.row_group(0)
   > <pyarrow._parquet.RowGroupMetaData object at 0x1375b9080>
   >   num_columns: 2
   >   num_rows: 6
   >   total_byte_size: 256
   > >>> writer.writer.metadata.row_group(0).to_dict()
   > {
   > 	'num_columns': 2,
   > 	'num_rows': 6,
   > 	'total_byte_size': 256,
   > 	'columns': [{
   > 		'file_offset': 119,
   > 		'file_path': '',
   > 		'physical_type': 'INT64',
   > 		'num_values': 6,
   > 		'path_in_schema': 'n_legs',
   > 		'is_stats_set': True,
   > 		'statistics': {
   > 			'has_min_max': True,
   > 			'min': 2,
   > 			'max': 100,
   > 			'null_count': 0,
   > 			'distinct_count': 0,
   > 			'num_values': 6,
   > 			'physical_type': 'INT64'
   > 		},
   > 		'compression': 'SNAPPY',
   > 		'encodings': ('RLE_DICTIONARY', 'PLAIN', 'RLE'),
   > 		'has_dictionary_page': True,
   > 		'dictionary_page_offset': 4,
   > 		'data_page_offset': 46,
   > 		'total_compressed_size': 115,
   > 		'total_uncompressed_size': 117
   > 	}, {
   > 		'file_offset': 359,
   > 		'file_path': '',
   > 		'physical_type': 'BYTE_ARRAY',
   > 		'num_values': 6,
   > 		'path_in_schema': 'animal',
   > 		'is_stats_set': True,
   > 		'statistics': {
   > 			'has_min_max': True,
   > 			'min': 'Brittle stars',
   > 			'max': 'Parrot',
   > 			'null_count': 0,
   > 			'distinct_count': 0,
   > 			'num_values': 6,
   > 			'physical_type': 'BYTE_ARRAY'
   > 		},
   > 		'compression': 'SNAPPY',
   > 		'encodings': ('RLE_DICTIONARY', 'PLAIN', 'RLE'),
   > 		'has_dictionary_page': True,
   > 		'dictionary_page_offset': 215,
   > 		'data_page_offset': 302,
   > 		'total_compressed_size': 144,
   > 		'total_uncompressed_size': 139
   > 	}]
   > }
   > ```
   
   Thanks for pointing that out, I completely missed the fact that you can get this data from the ParquetWriter after caling `close()`. That changes everything.


-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1285259053


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1039,335 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.values()).union({"INT96"})
+
+
+class StatsAggregator:
+    current_min: Any
+    current_max: Any
+    trunc_length: Optional[int]
+
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min = None
+        self.current_max = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        expected_physical_type = _PRIMITIVE_TO_PHYSICAL[iceberg_type]
+        if expected_physical_type != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {expected_physical_type}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        self.current_min = val if self.current_min is None else min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        self.current_max = val if self.current_max is None else max(val, self.current_max)
+
+    def get_min(self) -> bytes:
+        return self.serialize(
+            self.current_min
+            if self.trunc_length is None
+            else TruncateTransform(width=self.trunc_length).transform(self.primitive_type)(self.current_min)
+        )
+
+    def get_max(self) -> Optional[bytes]:
+        if self.current_max is None:
+            return None
+
+        if self.primitive_type == StringType():
+            if type(self.current_max) != str:
+                raise ValueError("Expected the current_max to be a string")
+
+            s_result = self.current_max[: self.trunc_length]
+            if s_result != self.current_max:
+                chars = [*s_result]
+
+                for i in range(-1, -len(s_result) - 1, -1):
+                    try:
+                        to_inc = ord(chars[i])
+                        # will raise exception if the highest unicode code is reached
+                        _next = chr(to_inc + 1)
+                        chars[i] = _next
+                        return self.serialize("".join(chars))
+                    except ValueError:
+                        pass
+                return None  # didn't find a valid upper bound
+            return self.serialize(s_result)
+        elif self.primitive_type == BinaryType():
+            if type(self.current_max) != bytes:
+                raise ValueError("Expected the current_max to be bytes")
+            b_result = self.current_max[: self.trunc_length]
+            if b_result != self.current_max:
+                _bytes = [*b_result]
+                for i in range(-1, -len(b_result) - 1, -1):
+                    if _bytes[i] < 255:
+                        _bytes[i] += 1
+                        return b"".join([i.to_bytes(1, byteorder="little") for i in _bytes])
+                return None
+
+            return self.serialize(b_result)
+        else:
+            return self.serialize(self.current_max)[: self.trunc_length]
+
+
+DEFAULT_TRUNCATION_LENGTH = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+
+class MetricModeTypes(Enum):
+    TRUNCATE = "truncate"
+    NONE = "none"
+    COUNTS = "counts"
+    FULL = "full"
+
+
+DEFAULT_METRICS_MODE_KEY = "write.metadata.metrics.default"
+COLUMN_METRICS_MODE_KEY_PREFIX = "write.metadata.metrics.column"
+
+
+@dataclass(frozen=True)
+class MetricsMode(Singleton):
+    type: MetricModeTypes
+    length: Optional[int] = None
+
+
+def match_metrics_mode(mode: str) -> MetricsMode:
+    sanitized_mode = mode.lower()
+    if sanitized_mode.startswith("truncate"):
+        m = re.match(TRUNCATION_EXPR, mode, re.IGNORECASE)
+        if m:
+            length = int(m[1])
+            if length < 1:
+                raise ValueError("Truncation length must be larger than 0")
+            return MetricsMode(MetricModeTypes.TRUNCATE, int(m[1]))
+        else:
+            raise ValueError(f"Malformed truncate: {mode}")
+    elif sanitized_mode.startswith("none"):
+        return MetricsMode(MetricModeTypes.NONE)
+    elif sanitized_mode.startswith("counts"):
+        return MetricsMode(MetricModeTypes.COUNTS)
+    elif sanitized_mode.startswith("full"):
+        return MetricsMode(MetricModeTypes.FULL)
+    else:
+        raise ValueError(f"Unsupported metrics mode: {mode}")
+
+
+@dataclass(frozen=True)
+class StatisticsCollector:
+    field_id: int
+    iceberg_type: PrimitiveType
+    mode: MetricsMode
+    column_name: str
+
+
+class PyArrowStatisticsCollector(PreOrderSchemaVisitor[List[StatisticsCollector]]):
+    _field_id: int = 0
+    _schema: Schema
+    _properties: Dict[str, str]
+
+    def __init__(self, schema: Schema, properties: Dict[str, str]):
+        self._schema = schema
+        self._properties = properties
+
+    def schema(self, schema: Schema, struct_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        return struct_result()
+
+    def struct(
+        self, struct: StructType, field_results: List[Callable[[], List[StatisticsCollector]]]
+    ) -> List[StatisticsCollector]:
+        return list(chain(*[result() for result in field_results]))
+
+    def field(self, field: NestedField, field_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = field.field_id

Review Comment:
   I don't like relying on pre-order traversal for keeping the field ID. This is exactly what the `before_field` and `after_field` callbacks are intended for in the normal `SchemaVisitor`, and using that doesn't require separate logic for map key/value and list element fields.
   
   The main issue is that the pre-order visitor doesn't actually perform a pre-order visit. It creates `Callable` results that the visitor can use to perform a pre-order visit. I think that makes this more complex, harder to use, and more likely to break.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1288443886


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1039,335 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.values()).union({"INT96"})
+
+
+class StatsAggregator:
+    current_min: Any
+    current_max: Any
+    trunc_length: Optional[int]
+
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min = None
+        self.current_max = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        expected_physical_type = _PRIMITIVE_TO_PHYSICAL[iceberg_type]
+        if expected_physical_type != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {expected_physical_type}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        self.current_min = val if self.current_min is None else min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        self.current_max = val if self.current_max is None else max(val, self.current_max)
+
+    def get_min(self) -> bytes:
+        return self.serialize(
+            self.current_min
+            if self.trunc_length is None
+            else TruncateTransform(width=self.trunc_length).transform(self.primitive_type)(self.current_min)
+        )
+
+    def get_max(self) -> Optional[bytes]:
+        if self.current_max is None:
+            return None
+
+        if self.primitive_type == StringType():
+            if type(self.current_max) != str:
+                raise ValueError("Expected the current_max to be a string")
+
+            s_result = self.current_max[: self.trunc_length]
+            if s_result != self.current_max:
+                chars = [*s_result]
+
+                for i in range(-1, -len(s_result) - 1, -1):
+                    try:
+                        to_inc = ord(chars[i])
+                        # will raise exception if the highest unicode code is reached
+                        _next = chr(to_inc + 1)
+                        chars[i] = _next
+                        return self.serialize("".join(chars))
+                    except ValueError:
+                        pass
+                return None  # didn't find a valid upper bound
+            return self.serialize(s_result)
+        elif self.primitive_type == BinaryType():
+            if type(self.current_max) != bytes:
+                raise ValueError("Expected the current_max to be bytes")
+            b_result = self.current_max[: self.trunc_length]
+            if b_result != self.current_max:
+                _bytes = [*b_result]
+                for i in range(-1, -len(b_result) - 1, -1):
+                    if _bytes[i] < 255:
+                        _bytes[i] += 1
+                        return b"".join([i.to_bytes(1, byteorder="little") for i in _bytes])
+                return None
+
+            return self.serialize(b_result)
+        else:
+            return self.serialize(self.current_max)[: self.trunc_length]
+
+
+DEFAULT_TRUNCATION_LENGTH = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+
+class MetricModeTypes(Enum):
+    TRUNCATE = "truncate"
+    NONE = "none"
+    COUNTS = "counts"
+    FULL = "full"
+
+
+DEFAULT_METRICS_MODE_KEY = "write.metadata.metrics.default"
+COLUMN_METRICS_MODE_KEY_PREFIX = "write.metadata.metrics.column"
+
+
+@dataclass(frozen=True)
+class MetricsMode(Singleton):
+    type: MetricModeTypes
+    length: Optional[int] = None
+
+
+def match_metrics_mode(mode: str) -> MetricsMode:
+    sanitized_mode = mode.lower()
+    if sanitized_mode.startswith("truncate"):
+        m = re.match(TRUNCATION_EXPR, mode, re.IGNORECASE)
+        if m:
+            length = int(m[1])
+            if length < 1:
+                raise ValueError("Truncation length must be larger than 0")
+            return MetricsMode(MetricModeTypes.TRUNCATE, int(m[1]))
+        else:
+            raise ValueError(f"Malformed truncate: {mode}")
+    elif sanitized_mode.startswith("none"):
+        return MetricsMode(MetricModeTypes.NONE)
+    elif sanitized_mode.startswith("counts"):
+        return MetricsMode(MetricModeTypes.COUNTS)
+    elif sanitized_mode.startswith("full"):
+        return MetricsMode(MetricModeTypes.FULL)
+    else:
+        raise ValueError(f"Unsupported metrics mode: {mode}")
+
+
+@dataclass(frozen=True)
+class StatisticsCollector:
+    field_id: int
+    iceberg_type: PrimitiveType
+    mode: MetricsMode
+    column_name: str
+
+
+class PyArrowStatisticsCollector(PreOrderSchemaVisitor[List[StatisticsCollector]]):
+    _field_id: int = 0
+    _schema: Schema
+    _properties: Dict[str, str]
+
+    def __init__(self, schema: Schema, properties: Dict[str, str]):
+        self._schema = schema
+        self._properties = properties
+
+    def schema(self, schema: Schema, struct_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        return struct_result()
+
+    def struct(
+        self, struct: StructType, field_results: List[Callable[[], List[StatisticsCollector]]]
+    ) -> List[StatisticsCollector]:
+        return list(chain(*[result() for result in field_results]))
+
+    def field(self, field: NestedField, field_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = field.field_id
+        result = field_result()
+        return result
+
+    def list(self, list_type: ListType, element_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = list_type.element_id
+        return element_result()
+
+    def map(
+        self,
+        map_type: MapType,
+        key_result: Callable[[], List[StatisticsCollector]],
+        value_result: Callable[[], List[StatisticsCollector]],
+    ) -> List[StatisticsCollector]:
+        self._field_id = map_type.key_id
+        k = key_result()
+        self._field_id = map_type.value_id
+        v = value_result()
+        return k + v
+
+    def primitive(self, primitive: PrimitiveType) -> List[StatisticsCollector]:
+        column_name = self._schema.find_column_name(self._field_id)
+        if column_name is None:
+            raise ValueError(f"Column for field {self._field_id} not found")
+
+        metrics_mode = MetricsMode(MetricModeTypes.TRUNCATE, DEFAULT_TRUNCATION_LENGTH)
+
+        default_mode = self._properties.get(DEFAULT_METRICS_MODE_KEY)
+        if default_mode:
+            metrics_mode = match_metrics_mode(default_mode)
+
+        col_mode = self._properties.get(f"{COLUMN_METRICS_MODE_KEY_PREFIX}.{column_name}")
+        if col_mode:
+            metrics_mode = match_metrics_mode(col_mode)
+
+        if (
+            not (isinstance(primitive, StringType) or isinstance(primitive, BinaryType))
+            and metrics_mode.type == MetricModeTypes.TRUNCATE
+        ):
+            metrics_mode = MetricsMode(MetricModeTypes.FULL)
+
+        return [StatisticsCollector(field_id=self._field_id, iceberg_type=primitive, mode=metrics_mode, column_name=column_name)]
+
+
+def fill_parquet_file_metadata(
+    df: DataFile,
+    parquet_metadata: pq.FileMetaData,
+    file_size: int,
+    table_metadata: TableMetadata,
+) -> None:
+    """
+    Computes and fills the following fields of the DataFile object.
+
+    - file_format
+    - record_count
+    - file_size_in_bytes
+    - column_sizes
+    - value_counts
+    - null_value_counts
+    - nan_value_counts
+    - lower_bounds
+    - upper_bounds
+    - split_offsets
+
+    Args:
+        df (DataFile): A DataFile object representing the Parquet file for which metadata is to be filled.
+        parquet_metadata (pyarrow.parquet.FileMetaData): A pyarrow metadata object.
+        file_size (int): The total compressed file size cannot be retrieved from the metadata and hence has to
+            be passed here. Depending on the kind of file system and pyarrow library call used, different
+            ways to obtain this value might be appropriate.
+        table_metadata (pyiceberg.table.metadata.TableMetadata): The Iceberg table metadata. It is required to
+            compute the mapping if column position to iceberg schema type id. It's also used to set the mode
+            for column metrics collection
+    """
+    schema = next(filter(lambda s: s.schema_id == table_metadata.current_schema_id, table_metadata.schemas))
+
+    stats_columns = pre_order_visit(schema, PyArrowStatisticsCollector(schema, table_metadata.properties))
+
+    if parquet_metadata.num_columns != len(stats_columns):
+        raise ValueError(
+            f"Number of columns in metadata ({len(stats_columns)}) is different from the number of columns in pyarrow table ({parquet_metadata.num_columns})"
+        )
+
+    column_sizes: Dict[int, int] = {}
+    value_counts: Dict[int, int] = {}
+    split_offsets: List[int] = []
+
+    null_value_counts: Dict[int, int] = {}
+    nan_value_counts: Dict[int, int] = {}
+
+    col_aggs = {}
+
+    for r in range(parquet_metadata.num_row_groups):
+        # References:
+        # https://github.com/apache/iceberg/blob/fc381a81a1fdb8f51a0637ca27cd30673bd7aad3/parquet/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java#L232
+        # https://github.com/apache/parquet-mr/blob/ac29db4611f86a07cc6877b416aa4b183e09b353/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/ColumnChunkMetaData.java#L184
+
+        row_group = parquet_metadata.row_group(r)
+
+        data_offset = row_group.column(0).data_page_offset
+        dictionary_offset = row_group.column(0).dictionary_page_offset
+
+        if row_group.column(0).has_dictionary_page and dictionary_offset < data_offset:
+            split_offsets.append(dictionary_offset)
+        else:
+            split_offsets.append(data_offset)
+
+        for pos, stats_col in enumerate(stats_columns):
+            field_id = stats_col.field_id
+
+            column = row_group.column(pos)

Review Comment:
   This would need some translation because PyArrow uses different naming:
   
   ![image](https://github.com/apache/iceberg/assets/1134248/f2d194fc-4ba5-4c0d-82bd-bee87da1737d)
   
   I haven't found any way to get PyArrow out of order.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1289038138


##########
python/tests/io/test_pyarrow.py:
##########
@@ -1345,3 +1374,655 @@ def test_pyarrow_wrap_fsspec(example_task: FileScanTask, table_schema_simple: Sc
 bar: [[1,2,3]]
 baz: [[true,false,null]]"""
     )
+
+
+def construct_test_table() -> Tuple[Any, Any, Union[TableMetadataV1, TableMetadataV2]]:
+    table_metadata = {
+        "format-version": 2,
+        "location": "s3://bucket/test/location",
+        "last-column-id": 7,
+        "current-schema-id": 0,
+        "schemas": [
+            {
+                "type": "struct",
+                "schema-id": 0,
+                "fields": [
+                    {"id": 1, "name": "strings", "required": False, "type": "string"},
+                    {"id": 2, "name": "floats", "required": False, "type": "float"},
+                    {
+                        "id": 3,
+                        "name": "list",
+                        "required": False,
+                        "type": {"type": "list", "element-id": 5, "element": "long", "element-required": False},
+                    },
+                    {
+                        "id": 4,
+                        "name": "maps",
+                        "required": False,
+                        "type": {
+                            "type": "map",
+                            "key-id": 6,
+                            "key": "long",
+                            "value-id": 7,
+                            "value": "long",
+                            "value-required": False,
+                        },
+                    },
+                ],
+            },
+        ],
+        "default-spec-id": 0,
+        "partition-specs": [{"spec-id": 0, "fields": []}],
+        "properties": {},
+    }
+
+    table_metadata = TableMetadataUtil.parse_obj(table_metadata)
+    arrow_schema = schema_to_pyarrow(table_metadata.schemas[0])
+
+    _strings = ["zzzzzzzzzzzzzzzzzzzz", "rrrrrrrrrrrrrrrrrrrr", None, "aaaaaaaaaaaaaaaaaaaa"]
+
+    _floats = [3.14, math.nan, 1.69, 100]
+
+    _list = [[1, 2, 3], [4, 5, 6], None, [7, 8, 9]]
+
+    _maps: List[Optional[Dict[int, int]]] = [
+        {1: 2, 3: 4},
+        None,
+        {5: 6},
+        {},
+    ]
+
+    table = pa.Table.from_pydict(
+        {
+            "strings": _strings,
+            "floats": _floats,
+            "list": _list,
+            "maps": _maps,
+        },
+        schema=arrow_schema,
+    )
+    metadata_collector: List[Any] = []
+
+    with pa.BufferOutputStream() as f:
+        with pq.ParquetWriter(f, table.schema, metadata_collector=metadata_collector) as writer:
+            writer.write_table(table)
+
+        return f.getvalue(), metadata_collector[0], table_metadata
+
+
+def test_record_count() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.record_count == 4
+
+
+def test_file_size() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.file_size_in_bytes == len(file_bytes)
+
+
+def test_value_counts() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.value_counts) == 5
+    assert datafile.value_counts[1] == 4
+    assert datafile.value_counts[2] == 4
+    assert datafile.value_counts[5] == 10  # 3 lists with 3 items and a None value
+    assert datafile.value_counts[6] == 5
+    assert datafile.value_counts[7] == 5
+
+
+def test_column_sizes() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.column_sizes) == 5
+    # these values are an artifact of how the write_table encodes the columns
+    assert datafile.column_sizes[1] == 116
+    assert datafile.column_sizes[2] == 89
+    assert datafile.column_sizes[5] == 151
+    assert datafile.column_sizes[6] == 117
+    assert datafile.column_sizes[7] == 117
+
+
+def test_null_and_nan_counts() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.null_value_counts) == 5
+    assert datafile.null_value_counts[1] == 1
+    assert datafile.null_value_counts[2] == 0
+    assert datafile.null_value_counts[5] == 1
+    assert datafile.null_value_counts[6] == 2
+    assert datafile.null_value_counts[7] == 2
+
+    # #arrow does not include this in the statistics
+    # assert len(datafile.nan_value_counts)  == 3
+    # assert datafile.nan_value_counts[1]    == 0

Review Comment:
   Opened an issue a while ago: https://github.com/apache/arrow/issues/36068



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] maxdebayser commented on pull request #7831: Python: Compute parquet stats

Posted by "maxdebayser (via GitHub)" <gi...@apache.org>.
maxdebayser commented on PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#issuecomment-1623803047

   @Fokko 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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] maxdebayser commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "maxdebayser (via GitHub)" <gi...@apache.org>.
maxdebayser commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1258886052


##########
python/pyiceberg/utils/file_stats.py:
##########
@@ -0,0 +1,333 @@
+#  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.
+
+import struct
+from typing import (
+    Any,
+    Dict,
+    List,
+    Union,
+)
+
+import pyarrow.lib
+import pyarrow.parquet as pq
+
+from pyiceberg.manifest import DataFile, FileFormat
+from pyiceberg.schema import Schema, SchemaVisitor, visit
+from pyiceberg.types import (
+    IcebergType,
+    ListType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StructType,
+)
+
+BOUND_TRUNCATED_LENGHT = 16
+
+# Serialization rules: https://iceberg.apache.org/spec/#binary-single-value-serialization
+#
+# Type      Binary serialization
+# boolean   0x00 for false, non-zero byte for true
+# int       Stored as 4-byte little-endian
+# long      Stored as 8-byte little-endian
+# float     Stored as 4-byte little-endian
+# double    Stored as 8-byte little-endian
+# date      Stores days from the 1970-01-01 in an 4-byte little-endian int
+# time      Stores microseconds from midnight in an 8-byte little-endian long
+# timestamp without zone	Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long
+# timestamp with zone	Stores microseconds from 1970-01-01 00:00:00.000000 UTC in an 8-byte little-endian long
+# string    UTF-8 bytes (without length)
+# uuid      16-byte big-endian value, see example in Appendix B
+# fixed(L)  Binary value
+# binary    Binary value (without length)
+#
+
+
+def bool_to_avro(value: bool) -> bytes:
+    return struct.pack("?", value)
+
+
+def int32_to_avro(value: int) -> bytes:
+    return struct.pack("<i", value)
+
+
+def int64_to_avro(value: int) -> bytes:
+    return struct.pack("<q", value)
+
+
+def float_to_avro(value: float) -> bytes:
+    return struct.pack("<f", value)
+
+
+def double_to_avro(value: float) -> bytes:
+    return struct.pack("<d", value)
+
+
+def bytes_to_avro(value: Union[bytes, str]) -> bytes:
+    if type(value) == str:
+        return value.encode()
+    else:
+        assert isinstance(value, bytes)  # appeases mypy
+        return value
+
+
+class StatsAggregator:
+    def __init__(self, type_string: str):
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.serialize: Any = None
+
+        if type_string == "BOOLEAN":
+            self.serialize = bool_to_avro
+        elif type_string == "INT32":
+            self.serialize = int32_to_avro
+        elif type_string == "INT64":
+            self.serialize = int64_to_avro
+        elif type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+        elif type_string == "FLOAT":
+            self.serialize = float_to_avro
+        elif type_string == "DOUBLE":
+            self.serialize = double_to_avro
+        elif type_string == "BYTE_ARRAY":
+            self.serialize = bytes_to_avro
+        elif type_string == "FIXED_LEN_BYTE_ARRAY":
+            self.serialize = bytes_to_avro
+        else:
+            raise AssertionError(f"Unknown physical type {type_string}")
+
+    def add_min(self, val: bytes) -> None:
+        if not self.current_min:
+            self.current_min = val
+        elif val < self.current_min:
+            self.current_min = val
+
+    def add_max(self, val: bytes) -> None:
+        if not self.current_max:
+            self.current_max = val
+        elif self.current_max < val:
+            self.current_max = val
+
+    def get_min(self) -> bytes:
+        return self.serialize(self.current_min)[:BOUND_TRUNCATED_LENGHT]
+
+    def get_max(self) -> bytes:
+        return self.serialize(self.current_max)[:BOUND_TRUNCATED_LENGHT]
+
+
+def fill_parquet_file_metadata(
+    df: DataFile, metadata: pq.FileMetaData, col_path_2_iceberg_id: Dict[str, int], file_size: int
+) -> None:
+    """
+    Computes and fills the following fields of the DataFile object.
+
+    - file_format
+    - record_count
+    - file_size_in_bytes
+    - column_sizes
+    - value_counts
+    - null_value_counts
+    - nan_value_counts
+    - lower_bounds
+    - upper_bounds
+    - split_offsets
+
+    Args:
+        df (DataFile): A DataFile object representing the Parquet file for which metadata is to be filled.
+        metadata (pyarrow.parquet.FileMetaData): A pyarrow metadata object.
+        col_path_2_iceberg_id: A mapping of column paths as in the `path_in_schema` attribute of the colum

Review Comment:
   Yes, this is why I've added the `parquet_schema_to_ids` function that uses the  SchemaVisitor. If you prefer we can pass the schema to the `fill_parquet_file_metadata` and compute the mapping internally instead of making the caller of the function do it. The reason I decomposed it was to make testing easier.
   
   The `parquet_schema_to_ids` uses the `path_in_schema` column metadata attribute instead of the position because I'm not 100% sure if I can assume anything about the position, whereas using the full column path should be safe.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1264727823


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1013,3 +1025,271 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+class StatsAggregator:
+    def __init__(self, type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.trunc_length = trunc_length
+        self.primitive_type: Optional[PrimitiveType] = None
+
+        if type_string == "BOOLEAN":
+            self.primitive_type = BooleanType()
+        elif type_string == "INT32":
+            self.primitive_type = IntegerType()
+        elif type_string == "INT64":
+            self.primitive_type = LongType()
+        elif type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+        elif type_string == "FLOAT":
+            self.primitive_type = FloatType()
+        elif type_string == "DOUBLE":
+            self.primitive_type = DoubleType()
+        elif type_string == "BYTE_ARRAY":
+            self.primitive_type = BinaryType()
+        elif type_string == "FIXED_LEN_BYTE_ARRAY":
+            self.primitive_type = BinaryType()
+        else:
+            raise AssertionError(f"Unknown physical type {type_string}")
+
+    def serialize(self, value: Any) -> bytes:
+        if type(value) == str:
+            value = value.encode()
+        assert self.primitive_type is not None  # appease mypy
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        if self.current_min is None:
+            self.current_min = val
+        else:
+            self.current_min = min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        if self.current_max is None:
+            self.current_max = val
+        else:
+            self.current_max = max(self.current_max, val)
+
+    def get_min(self) -> bytes:
+        return self.serialize(self.current_min)[: self.trunc_length]
+
+    def get_max(self) -> bytes:
+        return self.serialize(self.current_max)[: self.trunc_length]
+
+
+DEFAULT_TRUNCATION_LENGHT = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+
+class MetricModeTypes(Enum):
+    TRUNCATE = "truncate"
+    NONE = "none"
+    COUNTS = "counts"
+    FULL = "full"
+
+
+DEFAULT_METRICS_MODE_KEY = "write.metadata.metrics.default"
+COLUMN_METRICS_MODE_KEY = "write.metadata.metrics.column"
+
+
+@dataclass(frozen=True)
+class MetricsMode(Singleton):
+    type: MetricModeTypes
+    length: Optional[int] = None
+
+
+def match_metrics_mode(mode: str) -> MetricsMode:
+    m = re.match(TRUNCATION_EXPR, mode, re.IGNORECASE)
+    if m:
+        length = int(m[1])
+        if length < 1:
+            raise AssertionError("Truncation length must be larger than 0")
+        return MetricsMode(MetricModeTypes.TRUNCATE, int(m[1]))
+    elif re.match("^none$", mode, re.IGNORECASE):
+        return MetricsMode(MetricModeTypes.NONE)
+    elif re.match("^counts$", mode, re.IGNORECASE):
+        return MetricsMode(MetricModeTypes.COUNTS)
+    elif re.match("^full$", mode, re.IGNORECASE):
+        return MetricsMode(MetricModeTypes.FULL)
+    else:
+        raise AssertionError(f"Unsupported metrics mode {mode}")
+
+
+@dataclass(frozen=True)
+class StatisticsCollector:
+    field_id: int
+    iceberg_type: PrimitiveType
+    mode: MetricsMode
+    column_name: str
+
+
+class PyArrowStatisticsCollector(PreOrderSchemaVisitor[List[StatisticsCollector]]):
+    _field_id = 0
+    _schema: Schema
+    _properties: Dict[str, str]
+
+    def __init__(self, schema: Schema, properties: Dict[str, str]):
+        self._schema = schema
+        self._properties = properties
+
+    def schema(self, schema: Schema, struct_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        return struct_result()
+
+    def struct(
+        self, struct: StructType, field_results: List[Callable[[], List[StatisticsCollector]]]
+    ) -> List[StatisticsCollector]:
+        return list(chain(*[result() for result in field_results]))
+
+    def field(self, field: NestedField, field_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = field.field_id
+        result = field_result()
+        return result
+
+    def list(self, list_type: ListType, element_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = list_type.element_id
+        return element_result()
+
+    def map(
+        self,
+        map_type: MapType,
+        key_result: Callable[[], List[StatisticsCollector]],
+        value_result: Callable[[], List[StatisticsCollector]],
+    ) -> List[StatisticsCollector]:
+        self._field_id = map_type.key_id
+        k = key_result()
+        self._field_id = map_type.value_id
+        v = value_result()
+        return k + v
+
+    def primitive(self, primitive: PrimitiveType) -> List[StatisticsCollector]:
+        column_name = self._schema.find_column_name(self._field_id)
+        assert column_name is not None, f"Column for field {self._field_id} not found"
+
+        metrics_mode = MetricsMode(MetricModeTypes.TRUNCATE, DEFAULT_TRUNCATION_LENGHT)
+
+        default_mode = self._properties.get(DEFAULT_METRICS_MODE_KEY)
+        if default_mode:
+            metrics_mode = match_metrics_mode(default_mode)
+
+        col_mode = self._properties.get(f"{COLUMN_METRICS_MODE_KEY}.{column_name}")
+        if col_mode:
+            metrics_mode = match_metrics_mode(col_mode)
+
+        return [StatisticsCollector(field_id=self._field_id, iceberg_type=primitive, mode=metrics_mode, column_name=column_name)]
+
+
+def fill_parquet_file_metadata(
+    df: DataFile,
+    parquet_metadata: pq.FileMetaData,
+    file_size: int,
+    table_metadata: TableMetadata,
+) -> None:
+    """
+    Computes and fills the following fields of the DataFile object.
+
+    - file_format
+    - record_count
+    - file_size_in_bytes
+    - column_sizes
+    - value_counts
+    - null_value_counts
+    - nan_value_counts
+    - lower_bounds
+    - upper_bounds
+    - split_offsets
+
+    Args:
+        df (DataFile): A DataFile object representing the Parquet file for which metadata is to be filled.
+        parquet_metadata (pyarrow.parquet.FileMetaData): A pyarrow metadata object.
+        file_size (int): The total compressed file size cannot be retrieved from the metadata and hence has to
+            be passed here. Depending on the kind of file system and pyarrow library call used, different
+            ways to obtain this value might be appropriate.
+        table_metadata (pyiceberg.table.metadata.TableMetadata): The Iceberg table metadata. It is required to
+            compute the mapping if column position to iceberg schema type id. It's also used to set the mode
+            for column metrics collection
+    """
+    schema = next(filter(lambda s: s.schema_id == table_metadata.current_schema_id, table_metadata.schemas))
+
+    stats_columns = pre_order_visit(schema, PyArrowStatisticsCollector(schema, table_metadata.properties))
+    assert parquet_metadata.num_columns == len(
+        stats_columns
+    ), f"Number of columns in metadata ({len(stats_columns)}) is different from the number of columns in pyarrow table ({parquet_metadata.num_columns})"
+
+    column_sizes: Dict[int, int] = {}
+    value_counts: Dict[int, int] = {}
+    split_offsets: List[int] = []
+
+    null_value_counts: Dict[int, int] = {}
+    nan_value_counts: Dict[int, int] = {}
+
+    col_aggs = {}
+
+    for r in range(parquet_metadata.num_row_groups):
+        # References:
+        # https://github.com/apache/iceberg/blob/fc381a81a1fdb8f51a0637ca27cd30673bd7aad3/parquet/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java#L232
+        # https://github.com/apache/parquet-mr/blob/ac29db4611f86a07cc6877b416aa4b183e09b353/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/ColumnChunkMetaData.java#L184
+
+        row_group = parquet_metadata.row_group(r)
+
+        data_offset = row_group.column(0).data_page_offset
+        dictionary_offset = row_group.column(0).dictionary_page_offset
+
+        if row_group.column(0).has_dictionary_page and dictionary_offset < data_offset:
+            split_offsets.append(dictionary_offset)
+        else:
+            split_offsets.append(data_offset)
+
+        for c in range(parquet_metadata.num_columns):
+            col_id = stats_columns[c].field_id
+
+            column = row_group.column(c)
+
+            column_sizes[col_id] = column_sizes.get(col_id, 0) + column.total_compressed_size
+
+            metrics_mode = stats_columns[c].mode
+
+            if metrics_mode == MetricsMode(MetricModeTypes.NONE):
+                continue
+
+            value_counts[col_id] = value_counts.get(col_id, 0) + column.num_values
+
+            if column.is_stats_set:
+                try:
+                    statistics = column.statistics
+
+                    null_value_counts[col_id] = null_value_counts.get(col_id, 0) + statistics.null_count
+
+                    if metrics_mode == MetricsMode(MetricModeTypes.COUNTS):
+                        continue
+
+                    if col_id not in col_aggs:
+                        col_aggs[col_id] = StatsAggregator(statistics.physical_type, metrics_mode.length)
+
+                    col_aggs[col_id].add_min(statistics.min)
+                    col_aggs[col_id].add_max(statistics.max)

Review Comment:
   What do you think of the following? This way we have fewer lookups. Also, I think it is best to stick with the Iceberg naming (field-id over column-id).
   ```suggestion
           for pos, stats_col in enumerate(stats_columns):
               field_id = stats_col.field_id
   
               column = row_group.column(pos)
   
               column_sizes[field_id] = column_sizes.get(field_id, 0) + column.total_compressed_size
   
               if stats_col.mode == MetricsMode(MetricModeTypes.NONE):
                   continue
   
               value_counts[field_id] = value_counts.get(field_id, 0) + column.num_values
   
               if column.is_stats_set:
                   try:
                       statistics = column.statistics
   
                       null_value_counts[field_id] = null_value_counts.get(field_id, 0) + statistics.null_count
   
                       if stats_col.mode == MetricsMode(MetricModeTypes.COUNTS):
                           continue
   
                       if field_id not in col_aggs:
                           col_aggs[field_id] = StatsAggregator(stats_col.iceberg_type, stats_col.mode.length)
   
                       col_aggs[field_id].add_min(statistics.min)
                       col_aggs[field_id].add_max(statistics.max)
   ```



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1257807284


##########
python/pyiceberg/utils/file_stats.py:
##########
@@ -0,0 +1,333 @@
+#  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.
+
+import struct
+from typing import (
+    Any,
+    Dict,
+    List,
+    Union,
+)
+
+import pyarrow.lib
+import pyarrow.parquet as pq
+
+from pyiceberg.manifest import DataFile, FileFormat
+from pyiceberg.schema import Schema, SchemaVisitor, visit
+from pyiceberg.types import (
+    IcebergType,
+    ListType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StructType,
+)
+
+BOUND_TRUNCATED_LENGHT = 16
+
+# Serialization rules: https://iceberg.apache.org/spec/#binary-single-value-serialization
+#
+# Type      Binary serialization
+# boolean   0x00 for false, non-zero byte for true
+# int       Stored as 4-byte little-endian
+# long      Stored as 8-byte little-endian
+# float     Stored as 4-byte little-endian
+# double    Stored as 8-byte little-endian
+# date      Stores days from the 1970-01-01 in an 4-byte little-endian int
+# time      Stores microseconds from midnight in an 8-byte little-endian long
+# timestamp without zone	Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long
+# timestamp with zone	Stores microseconds from 1970-01-01 00:00:00.000000 UTC in an 8-byte little-endian long
+# string    UTF-8 bytes (without length)
+# uuid      16-byte big-endian value, see example in Appendix B
+# fixed(L)  Binary value
+# binary    Binary value (without length)
+#
+
+
+def bool_to_avro(value: bool) -> bytes:
+    return struct.pack("?", value)

Review Comment:
   Can we initialize the structs just once? Similar to the Avro reading:
   
   https://github.com/apache/iceberg/blob/e389e4d139624a49729379acd330dd9c96187b04/python/pyiceberg/avro/__init__.py#L19-L20



##########
python/pyiceberg/utils/file_stats.py:
##########
@@ -0,0 +1,333 @@
+#  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.
+
+import struct
+from typing import (
+    Any,
+    Dict,
+    List,
+    Union,
+)
+
+import pyarrow.lib
+import pyarrow.parquet as pq
+
+from pyiceberg.manifest import DataFile, FileFormat
+from pyiceberg.schema import Schema, SchemaVisitor, visit
+from pyiceberg.types import (
+    IcebergType,
+    ListType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StructType,
+)
+
+BOUND_TRUNCATED_LENGHT = 16

Review Comment:
   In Spark this is configurable, but I'm fine with leaving this as is right now.



##########
python/tests/utils/test_file_stats.py:
##########
@@ -0,0 +1,361 @@
+# 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.
+
+
+import math
+import struct
+from tempfile import TemporaryDirectory
+from typing import Any, List
+
+import pyarrow as pa
+import pyarrow.parquet as pq
+
+from pyiceberg.manifest import DataFile
+from pyiceberg.schema import Schema
+from pyiceberg.utils.file_stats import BOUND_TRUNCATED_LENGHT, fill_parquet_file_metadata, parquet_schema_to_ids
+
+
+def construct_test_table() -> pa.Buffer:
+    schema = pa.schema(
+        [pa.field("strings", pa.string()), pa.field("floats", pa.float64()), pa.field("list", pa.list_(pa.int64()))]

Review Comment:
   Can we also add a map here?



##########
python/pyiceberg/utils/file_stats.py:
##########
@@ -0,0 +1,333 @@
+#  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.
+
+import struct
+from typing import (
+    Any,
+    Dict,
+    List,
+    Union,
+)
+
+import pyarrow.lib
+import pyarrow.parquet as pq
+
+from pyiceberg.manifest import DataFile, FileFormat
+from pyiceberg.schema import Schema, SchemaVisitor, visit
+from pyiceberg.types import (
+    IcebergType,
+    ListType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StructType,
+)
+
+BOUND_TRUNCATED_LENGHT = 16
+
+# Serialization rules: https://iceberg.apache.org/spec/#binary-single-value-serialization
+#
+# Type      Binary serialization
+# boolean   0x00 for false, non-zero byte for true
+# int       Stored as 4-byte little-endian
+# long      Stored as 8-byte little-endian
+# float     Stored as 4-byte little-endian
+# double    Stored as 8-byte little-endian
+# date      Stores days from the 1970-01-01 in an 4-byte little-endian int
+# time      Stores microseconds from midnight in an 8-byte little-endian long
+# timestamp without zone	Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long
+# timestamp with zone	Stores microseconds from 1970-01-01 00:00:00.000000 UTC in an 8-byte little-endian long
+# string    UTF-8 bytes (without length)
+# uuid      16-byte big-endian value, see example in Appendix B
+# fixed(L)  Binary value
+# binary    Binary value (without length)
+#
+
+
+def bool_to_avro(value: bool) -> bytes:
+    return struct.pack("?", value)
+
+
+def int32_to_avro(value: int) -> bytes:
+    return struct.pack("<i", value)
+
+
+def int64_to_avro(value: int) -> bytes:
+    return struct.pack("<q", value)
+
+
+def float_to_avro(value: float) -> bytes:
+    return struct.pack("<f", value)
+
+
+def double_to_avro(value: float) -> bytes:
+    return struct.pack("<d", value)
+
+
+def bytes_to_avro(value: Union[bytes, str]) -> bytes:
+    if type(value) == str:
+        return value.encode()
+    else:
+        assert isinstance(value, bytes)  # appeases mypy
+        return value
+
+
+class StatsAggregator:
+    def __init__(self, type_string: str):
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.serialize: Any = None
+
+        if type_string == "BOOLEAN":
+            self.serialize = bool_to_avro
+        elif type_string == "INT32":
+            self.serialize = int32_to_avro
+        elif type_string == "INT64":
+            self.serialize = int64_to_avro
+        elif type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+        elif type_string == "FLOAT":
+            self.serialize = float_to_avro
+        elif type_string == "DOUBLE":
+            self.serialize = double_to_avro
+        elif type_string == "BYTE_ARRAY":
+            self.serialize = bytes_to_avro
+        elif type_string == "FIXED_LEN_BYTE_ARRAY":
+            self.serialize = bytes_to_avro
+        else:
+            raise AssertionError(f"Unknown physical type {type_string}")
+
+    def add_min(self, val: bytes) -> None:
+        if not self.current_min:
+            self.current_min = val
+        elif val < self.current_min:
+            self.current_min = val
+
+    def add_max(self, val: bytes) -> None:
+        if not self.current_max:
+            self.current_max = val
+        elif self.current_max < val:
+            self.current_max = val
+
+    def get_min(self) -> bytes:
+        return self.serialize(self.current_min)[:BOUND_TRUNCATED_LENGHT]
+
+    def get_max(self) -> bytes:
+        return self.serialize(self.current_max)[:BOUND_TRUNCATED_LENGHT]
+
+
+def fill_parquet_file_metadata(
+    df: DataFile, metadata: pq.FileMetaData, col_path_2_iceberg_id: Dict[str, int], file_size: int

Review Comment:
   Should we move this one to `pyarrow.py`? This uses PyArrow classes that might not be installed.



##########
python/pyiceberg/utils/file_stats.py:
##########
@@ -0,0 +1,333 @@
+#  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.
+
+import struct
+from typing import (
+    Any,
+    Dict,
+    List,
+    Union,
+)
+
+import pyarrow.lib
+import pyarrow.parquet as pq
+
+from pyiceberg.manifest import DataFile, FileFormat
+from pyiceberg.schema import Schema, SchemaVisitor, visit
+from pyiceberg.types import (
+    IcebergType,
+    ListType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StructType,
+)
+
+BOUND_TRUNCATED_LENGHT = 16
+
+# Serialization rules: https://iceberg.apache.org/spec/#binary-single-value-serialization
+#
+# Type      Binary serialization
+# boolean   0x00 for false, non-zero byte for true
+# int       Stored as 4-byte little-endian
+# long      Stored as 8-byte little-endian
+# float     Stored as 4-byte little-endian
+# double    Stored as 8-byte little-endian
+# date      Stores days from the 1970-01-01 in an 4-byte little-endian int
+# time      Stores microseconds from midnight in an 8-byte little-endian long
+# timestamp without zone	Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long
+# timestamp with zone	Stores microseconds from 1970-01-01 00:00:00.000000 UTC in an 8-byte little-endian long
+# string    UTF-8 bytes (without length)
+# uuid      16-byte big-endian value, see example in Appendix B
+# fixed(L)  Binary value
+# binary    Binary value (without length)
+#
+
+
+def bool_to_avro(value: bool) -> bytes:
+    return struct.pack("?", value)
+
+
+def int32_to_avro(value: int) -> bytes:
+    return struct.pack("<i", value)
+
+
+def int64_to_avro(value: int) -> bytes:
+    return struct.pack("<q", value)
+
+
+def float_to_avro(value: float) -> bytes:
+    return struct.pack("<f", value)
+
+
+def double_to_avro(value: float) -> bytes:
+    return struct.pack("<d", value)
+
+
+def bytes_to_avro(value: Union[bytes, str]) -> bytes:
+    if type(value) == str:
+        return value.encode()
+    else:
+        assert isinstance(value, bytes)  # appeases mypy
+        return value
+
+
+class StatsAggregator:
+    def __init__(self, type_string: str):
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.serialize: Any = None
+
+        if type_string == "BOOLEAN":
+            self.serialize = bool_to_avro
+        elif type_string == "INT32":
+            self.serialize = int32_to_avro
+        elif type_string == "INT64":
+            self.serialize = int64_to_avro
+        elif type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+        elif type_string == "FLOAT":
+            self.serialize = float_to_avro
+        elif type_string == "DOUBLE":
+            self.serialize = double_to_avro
+        elif type_string == "BYTE_ARRAY":
+            self.serialize = bytes_to_avro
+        elif type_string == "FIXED_LEN_BYTE_ARRAY":
+            self.serialize = bytes_to_avro
+        else:
+            raise AssertionError(f"Unknown physical type {type_string}")
+
+    def add_min(self, val: bytes) -> None:
+        if not self.current_min:
+            self.current_min = val
+        elif val < self.current_min:
+            self.current_min = val
+
+    def add_max(self, val: bytes) -> None:
+        if not self.current_max:

Review Comment:
   ```suggestion
           if self.current_max is not None:
   ```



##########
python/pyiceberg/utils/file_stats.py:
##########
@@ -0,0 +1,333 @@
+#  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.
+
+import struct
+from typing import (
+    Any,
+    Dict,
+    List,
+    Union,
+)
+
+import pyarrow.lib
+import pyarrow.parquet as pq
+
+from pyiceberg.manifest import DataFile, FileFormat
+from pyiceberg.schema import Schema, SchemaVisitor, visit
+from pyiceberg.types import (
+    IcebergType,
+    ListType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StructType,
+)
+
+BOUND_TRUNCATED_LENGHT = 16
+
+# Serialization rules: https://iceberg.apache.org/spec/#binary-single-value-serialization
+#
+# Type      Binary serialization
+# boolean   0x00 for false, non-zero byte for true
+# int       Stored as 4-byte little-endian
+# long      Stored as 8-byte little-endian
+# float     Stored as 4-byte little-endian
+# double    Stored as 8-byte little-endian
+# date      Stores days from the 1970-01-01 in an 4-byte little-endian int
+# time      Stores microseconds from midnight in an 8-byte little-endian long
+# timestamp without zone	Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long
+# timestamp with zone	Stores microseconds from 1970-01-01 00:00:00.000000 UTC in an 8-byte little-endian long
+# string    UTF-8 bytes (without length)
+# uuid      16-byte big-endian value, see example in Appendix B
+# fixed(L)  Binary value
+# binary    Binary value (without length)
+#
+
+
+def bool_to_avro(value: bool) -> bytes:
+    return struct.pack("?", value)
+
+
+def int32_to_avro(value: int) -> bytes:
+    return struct.pack("<i", value)
+
+
+def int64_to_avro(value: int) -> bytes:
+    return struct.pack("<q", value)
+
+
+def float_to_avro(value: float) -> bytes:
+    return struct.pack("<f", value)
+
+
+def double_to_avro(value: float) -> bytes:
+    return struct.pack("<d", value)
+
+
+def bytes_to_avro(value: Union[bytes, str]) -> bytes:
+    if type(value) == str:
+        return value.encode()
+    else:
+        assert isinstance(value, bytes)  # appeases mypy
+        return value
+
+
+class StatsAggregator:
+    def __init__(self, type_string: str):
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.serialize: Any = None
+
+        if type_string == "BOOLEAN":
+            self.serialize = bool_to_avro
+        elif type_string == "INT32":
+            self.serialize = int32_to_avro
+        elif type_string == "INT64":
+            self.serialize = int64_to_avro
+        elif type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+        elif type_string == "FLOAT":
+            self.serialize = float_to_avro
+        elif type_string == "DOUBLE":
+            self.serialize = double_to_avro
+        elif type_string == "BYTE_ARRAY":
+            self.serialize = bytes_to_avro
+        elif type_string == "FIXED_LEN_BYTE_ARRAY":
+            self.serialize = bytes_to_avro
+        else:
+            raise AssertionError(f"Unknown physical type {type_string}")
+
+    def add_min(self, val: bytes) -> None:
+        if not self.current_min:
+            self.current_min = val
+        elif val < self.current_min:
+            self.current_min = val
+
+    def add_max(self, val: bytes) -> None:
+        if not self.current_max:
+            self.current_max = val
+        elif self.current_max < val:
+            self.current_max = val
+
+    def get_min(self) -> bytes:
+        return self.serialize(self.current_min)[:BOUND_TRUNCATED_LENGHT]
+
+    def get_max(self) -> bytes:
+        return self.serialize(self.current_max)[:BOUND_TRUNCATED_LENGHT]
+
+
+def fill_parquet_file_metadata(
+    df: DataFile, metadata: pq.FileMetaData, col_path_2_iceberg_id: Dict[str, int], file_size: int
+) -> None:
+    """
+    Computes and fills the following fields of the DataFile object.
+
+    - file_format
+    - record_count
+    - file_size_in_bytes
+    - column_sizes
+    - value_counts
+    - null_value_counts
+    - nan_value_counts
+    - lower_bounds
+    - upper_bounds
+    - split_offsets
+
+    Args:
+        df (DataFile): A DataFile object representing the Parquet file for which metadata is to be filled.
+        metadata (pyarrow.parquet.FileMetaData): A pyarrow metadata object.
+        col_path_2_iceberg_id: A mapping of column paths as in the `path_in_schema` attribute of the colum
+            metadata to iceberg schema IDs. For scalar columns this will be the column name. For complex types
+            it could be something like `my_map.key_value.value`
+        file_size (int): The total compressed file size cannot be retrieved from the metadata and hence has to
+            be passed here. Depending on the kind of file system and pyarrow library call used, different
+            ways to obtain this value might be appropriate.
+    """
+    col_index_2_id = {}
+
+    col_names = set(metadata.schema.names)
+
+    first_group = metadata.row_group(0)
+
+    for c in range(metadata.num_columns):
+        column = first_group.column(c)
+        col_path = column.path_in_schema
+
+        if col_path in col_path_2_iceberg_id:
+            col_index_2_id[c] = col_path_2_iceberg_id[col_path]
+        else:
+            raise AssertionError(f"Column path {col_path} couldn't be mapped to an iceberg ID")
+
+    column_sizes: Dict[int, int] = {}
+    value_counts: Dict[int, int] = {}
+    split_offsets: List[int] = []
+
+    null_value_counts: Dict[int, int] = {}
+    nan_value_counts: Dict[int, int] = {}
+
+    col_aggs = {}
+
+    for r in range(metadata.num_row_groups):
+        # References:
+        # https://github.com/apache/iceberg/blob/fc381a81a1fdb8f51a0637ca27cd30673bd7aad3/parquet/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java#L232
+        # https://github.com/apache/parquet-mr/blob/ac29db4611f86a07cc6877b416aa4b183e09b353/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/ColumnChunkMetaData.java#L184
+
+        row_group = metadata.row_group(r)
+
+        data_offset = row_group.column(0).data_page_offset
+        dictionary_offset = row_group.column(0).dictionary_page_offset
+
+        if row_group.column(0).has_dictionary_page and dictionary_offset < data_offset:
+            split_offsets.append(dictionary_offset)
+        else:
+            split_offsets.append(data_offset)
+
+        for c in range(metadata.num_columns):
+            col_id = col_index_2_id[c]
+
+            column = row_group.column(c)
+
+            column_sizes[col_id] = column_sizes.get(col_id, 0) + column.total_compressed_size
+            value_counts[col_id] = value_counts.get(col_id, 0) + column.num_values
+
+            if column.is_stats_set:
+                try:
+                    statistics = column.statistics
+
+                    null_value_counts[col_id] = null_value_counts.get(col_id, 0) + statistics.null_count
+
+                    if column.path_in_schema in col_names:
+                        # Iceberg seems to only have statistics for scalar columns
+
+                        if col_id not in col_aggs:
+                            col_aggs[col_id] = StatsAggregator(statistics.physical_type)
+
+                        col_aggs[col_id].add_min(statistics.min)

Review Comment:
   Do we need the intermediate `col_aggs`? I would prefer to directly write it into `{lower,upper}_bounds`.



##########
python/pyiceberg/utils/file_stats.py:
##########
@@ -0,0 +1,333 @@
+#  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.
+
+import struct
+from typing import (
+    Any,
+    Dict,
+    List,
+    Union,
+)
+
+import pyarrow.lib
+import pyarrow.parquet as pq
+
+from pyiceberg.manifest import DataFile, FileFormat
+from pyiceberg.schema import Schema, SchemaVisitor, visit
+from pyiceberg.types import (
+    IcebergType,
+    ListType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StructType,
+)
+
+BOUND_TRUNCATED_LENGHT = 16
+
+# Serialization rules: https://iceberg.apache.org/spec/#binary-single-value-serialization
+#
+# Type      Binary serialization
+# boolean   0x00 for false, non-zero byte for true
+# int       Stored as 4-byte little-endian
+# long      Stored as 8-byte little-endian
+# float     Stored as 4-byte little-endian
+# double    Stored as 8-byte little-endian
+# date      Stores days from the 1970-01-01 in an 4-byte little-endian int
+# time      Stores microseconds from midnight in an 8-byte little-endian long
+# timestamp without zone	Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long
+# timestamp with zone	Stores microseconds from 1970-01-01 00:00:00.000000 UTC in an 8-byte little-endian long
+# string    UTF-8 bytes (without length)
+# uuid      16-byte big-endian value, see example in Appendix B
+# fixed(L)  Binary value
+# binary    Binary value (without length)
+#
+
+
+def bool_to_avro(value: bool) -> bytes:
+    return struct.pack("?", value)
+
+
+def int32_to_avro(value: int) -> bytes:
+    return struct.pack("<i", value)
+
+
+def int64_to_avro(value: int) -> bytes:
+    return struct.pack("<q", value)
+
+
+def float_to_avro(value: float) -> bytes:
+    return struct.pack("<f", value)
+
+
+def double_to_avro(value: float) -> bytes:
+    return struct.pack("<d", value)
+
+
+def bytes_to_avro(value: Union[bytes, str]) -> bytes:
+    if type(value) == str:
+        return value.encode()
+    else:
+        assert isinstance(value, bytes)  # appeases mypy
+        return value
+
+
+class StatsAggregator:
+    def __init__(self, type_string: str):
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.serialize: Any = None
+
+        if type_string == "BOOLEAN":
+            self.serialize = bool_to_avro
+        elif type_string == "INT32":
+            self.serialize = int32_to_avro
+        elif type_string == "INT64":
+            self.serialize = int64_to_avro
+        elif type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+        elif type_string == "FLOAT":
+            self.serialize = float_to_avro
+        elif type_string == "DOUBLE":
+            self.serialize = double_to_avro
+        elif type_string == "BYTE_ARRAY":
+            self.serialize = bytes_to_avro
+        elif type_string == "FIXED_LEN_BYTE_ARRAY":
+            self.serialize = bytes_to_avro
+        else:
+            raise AssertionError(f"Unknown physical type {type_string}")
+
+    def add_min(self, val: bytes) -> None:
+        if not self.current_min:
+            self.current_min = val
+        elif val < self.current_min:
+            self.current_min = val
+
+    def add_max(self, val: bytes) -> None:
+        if not self.current_max:
+            self.current_max = val
+        elif self.current_max < val:
+            self.current_max = val
+
+    def get_min(self) -> bytes:
+        return self.serialize(self.current_min)[:BOUND_TRUNCATED_LENGHT]
+
+    def get_max(self) -> bytes:
+        return self.serialize(self.current_max)[:BOUND_TRUNCATED_LENGHT]
+
+
+def fill_parquet_file_metadata(
+    df: DataFile, metadata: pq.FileMetaData, col_path_2_iceberg_id: Dict[str, int], file_size: int
+) -> None:
+    """
+    Computes and fills the following fields of the DataFile object.
+
+    - file_format
+    - record_count
+    - file_size_in_bytes
+    - column_sizes
+    - value_counts
+    - null_value_counts
+    - nan_value_counts
+    - lower_bounds
+    - upper_bounds
+    - split_offsets
+
+    Args:
+        df (DataFile): A DataFile object representing the Parquet file for which metadata is to be filled.
+        metadata (pyarrow.parquet.FileMetaData): A pyarrow metadata object.
+        col_path_2_iceberg_id: A mapping of column paths as in the `path_in_schema` attribute of the colum
+            metadata to iceberg schema IDs. For scalar columns this will be the column name. For complex types
+            it could be something like `my_map.key_value.value`
+        file_size (int): The total compressed file size cannot be retrieved from the metadata and hence has to
+            be passed here. Depending on the kind of file system and pyarrow library call used, different
+            ways to obtain this value might be appropriate.
+    """
+    col_index_2_id = {}
+
+    col_names = set(metadata.schema.names)
+
+    first_group = metadata.row_group(0)
+
+    for c in range(metadata.num_columns):
+        column = first_group.column(c)
+        col_path = column.path_in_schema
+
+        if col_path in col_path_2_iceberg_id:
+            col_index_2_id[c] = col_path_2_iceberg_id[col_path]
+        else:
+            raise AssertionError(f"Column path {col_path} couldn't be mapped to an iceberg ID")
+
+    column_sizes: Dict[int, int] = {}
+    value_counts: Dict[int, int] = {}
+    split_offsets: List[int] = []
+
+    null_value_counts: Dict[int, int] = {}
+    nan_value_counts: Dict[int, int] = {}
+
+    col_aggs = {}
+
+    for r in range(metadata.num_row_groups):
+        # References:
+        # https://github.com/apache/iceberg/blob/fc381a81a1fdb8f51a0637ca27cd30673bd7aad3/parquet/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java#L232
+        # https://github.com/apache/parquet-mr/blob/ac29db4611f86a07cc6877b416aa4b183e09b353/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/ColumnChunkMetaData.java#L184
+
+        row_group = metadata.row_group(r)
+
+        data_offset = row_group.column(0).data_page_offset
+        dictionary_offset = row_group.column(0).dictionary_page_offset
+
+        if row_group.column(0).has_dictionary_page and dictionary_offset < data_offset:
+            split_offsets.append(dictionary_offset)
+        else:
+            split_offsets.append(data_offset)
+
+        for c in range(metadata.num_columns):
+            col_id = col_index_2_id[c]
+
+            column = row_group.column(c)
+
+            column_sizes[col_id] = column_sizes.get(col_id, 0) + column.total_compressed_size
+            value_counts[col_id] = value_counts.get(col_id, 0) + column.num_values
+
+            if column.is_stats_set:
+                try:
+                    statistics = column.statistics
+
+                    null_value_counts[col_id] = null_value_counts.get(col_id, 0) + statistics.null_count
+
+                    if column.path_in_schema in col_names:
+                        # Iceberg seems to only have statistics for scalar columns
+
+                        if col_id not in col_aggs:
+                            col_aggs[col_id] = StatsAggregator(statistics.physical_type)
+
+                        col_aggs[col_id].add_min(statistics.min)
+                        col_aggs[col_id].add_max(statistics.max)
+
+                except pyarrow.lib.ArrowNotImplementedError:
+                    pass

Review Comment:
   Should we log a warning here?



##########
python/pyiceberg/utils/file_stats.py:
##########
@@ -0,0 +1,333 @@
+#  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.
+
+import struct
+from typing import (
+    Any,
+    Dict,
+    List,
+    Union,
+)
+
+import pyarrow.lib
+import pyarrow.parquet as pq
+
+from pyiceberg.manifest import DataFile, FileFormat
+from pyiceberg.schema import Schema, SchemaVisitor, visit
+from pyiceberg.types import (
+    IcebergType,
+    ListType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StructType,
+)
+
+BOUND_TRUNCATED_LENGHT = 16
+
+# Serialization rules: https://iceberg.apache.org/spec/#binary-single-value-serialization
+#
+# Type      Binary serialization
+# boolean   0x00 for false, non-zero byte for true
+# int       Stored as 4-byte little-endian
+# long      Stored as 8-byte little-endian
+# float     Stored as 4-byte little-endian
+# double    Stored as 8-byte little-endian
+# date      Stores days from the 1970-01-01 in an 4-byte little-endian int
+# time      Stores microseconds from midnight in an 8-byte little-endian long
+# timestamp without zone	Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long
+# timestamp with zone	Stores microseconds from 1970-01-01 00:00:00.000000 UTC in an 8-byte little-endian long
+# string    UTF-8 bytes (without length)
+# uuid      16-byte big-endian value, see example in Appendix B
+# fixed(L)  Binary value
+# binary    Binary value (without length)
+#
+
+
+def bool_to_avro(value: bool) -> bytes:
+    return struct.pack("?", value)
+
+
+def int32_to_avro(value: int) -> bytes:
+    return struct.pack("<i", value)
+
+
+def int64_to_avro(value: int) -> bytes:
+    return struct.pack("<q", value)
+
+
+def float_to_avro(value: float) -> bytes:
+    return struct.pack("<f", value)
+
+
+def double_to_avro(value: float) -> bytes:
+    return struct.pack("<d", value)
+
+
+def bytes_to_avro(value: Union[bytes, str]) -> bytes:
+    if type(value) == str:
+        return value.encode()
+    else:
+        assert isinstance(value, bytes)  # appeases mypy
+        return value
+
+
+class StatsAggregator:
+    def __init__(self, type_string: str):
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.serialize: Any = None
+
+        if type_string == "BOOLEAN":
+            self.serialize = bool_to_avro
+        elif type_string == "INT32":
+            self.serialize = int32_to_avro
+        elif type_string == "INT64":
+            self.serialize = int64_to_avro
+        elif type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+        elif type_string == "FLOAT":
+            self.serialize = float_to_avro
+        elif type_string == "DOUBLE":
+            self.serialize = double_to_avro
+        elif type_string == "BYTE_ARRAY":
+            self.serialize = bytes_to_avro
+        elif type_string == "FIXED_LEN_BYTE_ARRAY":
+            self.serialize = bytes_to_avro
+        else:
+            raise AssertionError(f"Unknown physical type {type_string}")
+
+    def add_min(self, val: bytes) -> None:
+        if not self.current_min:
+            self.current_min = val
+        elif val < self.current_min:
+            self.current_min = val
+
+    def add_max(self, val: bytes) -> None:
+        if not self.current_max:
+            self.current_max = val
+        elif self.current_max < val:
+            self.current_max = val
+
+    def get_min(self) -> bytes:
+        return self.serialize(self.current_min)[:BOUND_TRUNCATED_LENGHT]
+
+    def get_max(self) -> bytes:
+        return self.serialize(self.current_max)[:BOUND_TRUNCATED_LENGHT]
+
+
+def fill_parquet_file_metadata(
+    df: DataFile, metadata: pq.FileMetaData, col_path_2_iceberg_id: Dict[str, int], file_size: int
+) -> None:
+    """
+    Computes and fills the following fields of the DataFile object.
+
+    - file_format
+    - record_count
+    - file_size_in_bytes
+    - column_sizes
+    - value_counts
+    - null_value_counts
+    - nan_value_counts
+    - lower_bounds
+    - upper_bounds
+    - split_offsets
+
+    Args:
+        df (DataFile): A DataFile object representing the Parquet file for which metadata is to be filled.
+        metadata (pyarrow.parquet.FileMetaData): A pyarrow metadata object.
+        col_path_2_iceberg_id: A mapping of column paths as in the `path_in_schema` attribute of the colum

Review Comment:
   A suggestion. Since we have the Iceberg write schema, we could also easily construct a `List[int, int]` that will tell the mapping of position to field-id. We have the `PreOrderSchemaVisitor` where we traverse the schema in order to construct this list. I don't like the `key_value` specific to PyArrow, and the list will be faster. WDYT?



##########
python/pyiceberg/utils/file_stats.py:
##########
@@ -0,0 +1,333 @@
+#  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.
+
+import struct
+from typing import (
+    Any,
+    Dict,
+    List,
+    Union,
+)
+
+import pyarrow.lib
+import pyarrow.parquet as pq
+
+from pyiceberg.manifest import DataFile, FileFormat
+from pyiceberg.schema import Schema, SchemaVisitor, visit
+from pyiceberg.types import (
+    IcebergType,
+    ListType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StructType,
+)
+
+BOUND_TRUNCATED_LENGHT = 16
+
+# Serialization rules: https://iceberg.apache.org/spec/#binary-single-value-serialization
+#
+# Type      Binary serialization
+# boolean   0x00 for false, non-zero byte for true
+# int       Stored as 4-byte little-endian
+# long      Stored as 8-byte little-endian
+# float     Stored as 4-byte little-endian
+# double    Stored as 8-byte little-endian
+# date      Stores days from the 1970-01-01 in an 4-byte little-endian int
+# time      Stores microseconds from midnight in an 8-byte little-endian long
+# timestamp without zone	Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long
+# timestamp with zone	Stores microseconds from 1970-01-01 00:00:00.000000 UTC in an 8-byte little-endian long
+# string    UTF-8 bytes (without length)
+# uuid      16-byte big-endian value, see example in Appendix B
+# fixed(L)  Binary value
+# binary    Binary value (without length)
+#
+
+
+def bool_to_avro(value: bool) -> bytes:
+    return struct.pack("?", value)
+
+
+def int32_to_avro(value: int) -> bytes:
+    return struct.pack("<i", value)
+
+
+def int64_to_avro(value: int) -> bytes:
+    return struct.pack("<q", value)
+
+
+def float_to_avro(value: float) -> bytes:
+    return struct.pack("<f", value)
+
+
+def double_to_avro(value: float) -> bytes:
+    return struct.pack("<d", value)
+
+
+def bytes_to_avro(value: Union[bytes, str]) -> bytes:
+    if type(value) == str:
+        return value.encode()
+    else:
+        assert isinstance(value, bytes)  # appeases mypy
+        return value
+
+
+class StatsAggregator:
+    def __init__(self, type_string: str):

Review Comment:
   Why are we using a `type_string` here? The PyIceberg `PrimitiveType` seems to do the trick for me. We have methods for converting a PyArrow type to an IcebergType.
   
   Nit:
   ```suggestion
       def __init__(self, type_string: str) -> None:
   ```



##########
python/pyiceberg/utils/file_stats.py:
##########
@@ -0,0 +1,333 @@
+#  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.
+
+import struct
+from typing import (
+    Any,
+    Dict,
+    List,
+    Union,
+)
+
+import pyarrow.lib
+import pyarrow.parquet as pq
+
+from pyiceberg.manifest import DataFile, FileFormat
+from pyiceberg.schema import Schema, SchemaVisitor, visit
+from pyiceberg.types import (
+    IcebergType,
+    ListType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StructType,
+)
+
+BOUND_TRUNCATED_LENGHT = 16
+
+# Serialization rules: https://iceberg.apache.org/spec/#binary-single-value-serialization
+#
+# Type      Binary serialization
+# boolean   0x00 for false, non-zero byte for true
+# int       Stored as 4-byte little-endian
+# long      Stored as 8-byte little-endian
+# float     Stored as 4-byte little-endian
+# double    Stored as 8-byte little-endian
+# date      Stores days from the 1970-01-01 in an 4-byte little-endian int
+# time      Stores microseconds from midnight in an 8-byte little-endian long
+# timestamp without zone	Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long
+# timestamp with zone	Stores microseconds from 1970-01-01 00:00:00.000000 UTC in an 8-byte little-endian long
+# string    UTF-8 bytes (without length)
+# uuid      16-byte big-endian value, see example in Appendix B
+# fixed(L)  Binary value
+# binary    Binary value (without length)
+#
+
+
+def bool_to_avro(value: bool) -> bytes:
+    return struct.pack("?", value)
+
+
+def int32_to_avro(value: int) -> bytes:
+    return struct.pack("<i", value)
+
+
+def int64_to_avro(value: int) -> bytes:
+    return struct.pack("<q", value)
+
+
+def float_to_avro(value: float) -> bytes:
+    return struct.pack("<f", value)
+
+
+def double_to_avro(value: float) -> bytes:
+    return struct.pack("<d", value)
+
+
+def bytes_to_avro(value: Union[bytes, str]) -> bytes:
+    if type(value) == str:
+        return value.encode()
+    else:
+        assert isinstance(value, bytes)  # appeases mypy
+        return value
+
+
+class StatsAggregator:
+    def __init__(self, type_string: str):
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.serialize: Any = None
+
+        if type_string == "BOOLEAN":
+            self.serialize = bool_to_avro
+        elif type_string == "INT32":
+            self.serialize = int32_to_avro
+        elif type_string == "INT64":
+            self.serialize = int64_to_avro
+        elif type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+        elif type_string == "FLOAT":
+            self.serialize = float_to_avro
+        elif type_string == "DOUBLE":
+            self.serialize = double_to_avro
+        elif type_string == "BYTE_ARRAY":
+            self.serialize = bytes_to_avro
+        elif type_string == "FIXED_LEN_BYTE_ARRAY":
+            self.serialize = bytes_to_avro
+        else:
+            raise AssertionError(f"Unknown physical type {type_string}")
+
+    def add_min(self, val: bytes) -> None:
+        if not self.current_min:
+            self.current_min = val
+        elif val < self.current_min:

Review Comment:
   Any reason to not use Python's build in `min` function? That one might be passed down to C.



##########
python/pyiceberg/utils/file_stats.py:
##########
@@ -0,0 +1,333 @@
+#  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.
+
+import struct
+from typing import (
+    Any,
+    Dict,
+    List,
+    Union,
+)
+
+import pyarrow.lib
+import pyarrow.parquet as pq
+
+from pyiceberg.manifest import DataFile, FileFormat
+from pyiceberg.schema import Schema, SchemaVisitor, visit
+from pyiceberg.types import (
+    IcebergType,
+    ListType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StructType,
+)
+
+BOUND_TRUNCATED_LENGHT = 16
+
+# Serialization rules: https://iceberg.apache.org/spec/#binary-single-value-serialization
+#
+# Type      Binary serialization
+# boolean   0x00 for false, non-zero byte for true
+# int       Stored as 4-byte little-endian
+# long      Stored as 8-byte little-endian
+# float     Stored as 4-byte little-endian
+# double    Stored as 8-byte little-endian
+# date      Stores days from the 1970-01-01 in an 4-byte little-endian int
+# time      Stores microseconds from midnight in an 8-byte little-endian long
+# timestamp without zone	Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long
+# timestamp with zone	Stores microseconds from 1970-01-01 00:00:00.000000 UTC in an 8-byte little-endian long
+# string    UTF-8 bytes (without length)
+# uuid      16-byte big-endian value, see example in Appendix B
+# fixed(L)  Binary value
+# binary    Binary value (without length)
+#
+
+
+def bool_to_avro(value: bool) -> bytes:
+    return struct.pack("?", value)
+
+
+def int32_to_avro(value: int) -> bytes:
+    return struct.pack("<i", value)
+
+
+def int64_to_avro(value: int) -> bytes:
+    return struct.pack("<q", value)
+
+
+def float_to_avro(value: float) -> bytes:
+    return struct.pack("<f", value)
+
+
+def double_to_avro(value: float) -> bytes:
+    return struct.pack("<d", value)
+
+
+def bytes_to_avro(value: Union[bytes, str]) -> bytes:
+    if type(value) == str:
+        return value.encode()
+    else:
+        assert isinstance(value, bytes)  # appeases mypy
+        return value
+
+
+class StatsAggregator:
+    def __init__(self, type_string: str):
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.serialize: Any = None
+
+        if type_string == "BOOLEAN":
+            self.serialize = bool_to_avro
+        elif type_string == "INT32":
+            self.serialize = int32_to_avro
+        elif type_string == "INT64":
+            self.serialize = int64_to_avro
+        elif type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+        elif type_string == "FLOAT":
+            self.serialize = float_to_avro
+        elif type_string == "DOUBLE":
+            self.serialize = double_to_avro
+        elif type_string == "BYTE_ARRAY":
+            self.serialize = bytes_to_avro
+        elif type_string == "FIXED_LEN_BYTE_ARRAY":
+            self.serialize = bytes_to_avro
+        else:
+            raise AssertionError(f"Unknown physical type {type_string}")
+
+    def add_min(self, val: bytes) -> None:
+        if not self.current_min:

Review Comment:
   ```suggestion
           if self.current_min is not None:
   ```



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] maxdebayser commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "maxdebayser (via GitHub)" <gi...@apache.org>.
maxdebayser commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1258820569


##########
python/pyiceberg/utils/file_stats.py:
##########
@@ -0,0 +1,333 @@
+#  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.
+
+import struct
+from typing import (
+    Any,
+    Dict,
+    List,
+    Union,
+)
+
+import pyarrow.lib
+import pyarrow.parquet as pq
+
+from pyiceberg.manifest import DataFile, FileFormat
+from pyiceberg.schema import Schema, SchemaVisitor, visit
+from pyiceberg.types import (
+    IcebergType,
+    ListType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StructType,
+)
+
+BOUND_TRUNCATED_LENGHT = 16

Review Comment:
   Thanks for pointing me in the right direction here, I've added a feature to configure this via table properties as documented here: https://iceberg.apache.org/docs/latest/configuration/



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] maxdebayser commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "maxdebayser (via GitHub)" <gi...@apache.org>.
maxdebayser commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1258806584


##########
python/pyiceberg/utils/file_stats.py:
##########
@@ -0,0 +1,333 @@
+#  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.
+
+import struct
+from typing import (
+    Any,
+    Dict,
+    List,
+    Union,
+)
+
+import pyarrow.lib
+import pyarrow.parquet as pq
+
+from pyiceberg.manifest import DataFile, FileFormat
+from pyiceberg.schema import Schema, SchemaVisitor, visit
+from pyiceberg.types import (
+    IcebergType,
+    ListType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StructType,
+)
+
+BOUND_TRUNCATED_LENGHT = 16
+
+# Serialization rules: https://iceberg.apache.org/spec/#binary-single-value-serialization
+#
+# Type      Binary serialization
+# boolean   0x00 for false, non-zero byte for true
+# int       Stored as 4-byte little-endian
+# long      Stored as 8-byte little-endian
+# float     Stored as 4-byte little-endian
+# double    Stored as 8-byte little-endian
+# date      Stores days from the 1970-01-01 in an 4-byte little-endian int
+# time      Stores microseconds from midnight in an 8-byte little-endian long
+# timestamp without zone	Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long
+# timestamp with zone	Stores microseconds from 1970-01-01 00:00:00.000000 UTC in an 8-byte little-endian long
+# string    UTF-8 bytes (without length)
+# uuid      16-byte big-endian value, see example in Appendix B
+# fixed(L)  Binary value
+# binary    Binary value (without length)
+#
+
+
+def bool_to_avro(value: bool) -> bytes:
+    return struct.pack("?", value)
+
+
+def int32_to_avro(value: int) -> bytes:
+    return struct.pack("<i", value)
+
+
+def int64_to_avro(value: int) -> bytes:
+    return struct.pack("<q", value)
+
+
+def float_to_avro(value: float) -> bytes:
+    return struct.pack("<f", value)
+
+
+def double_to_avro(value: float) -> bytes:
+    return struct.pack("<d", value)
+
+
+def bytes_to_avro(value: Union[bytes, str]) -> bytes:
+    if type(value) == str:
+        return value.encode()
+    else:
+        assert isinstance(value, bytes)  # appeases mypy
+        return value
+
+
+class StatsAggregator:
+    def __init__(self, type_string: str):

Review Comment:
   Because the "physical_type" attribute of the metadata is a parquet-specific string: https://github.com/apache/arrow/blob/d676078c13a02ad920eeea2acd5fa517f14526e2/cpp/src/parquet/parquet.thrift#L34 . It seems to me that it is a low-level implementation detail that ends up "leaking" up to the python client and that there is no API in pyarrow to handle this, but I might be wrong.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] maxdebayser commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "maxdebayser (via GitHub)" <gi...@apache.org>.
maxdebayser commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1260009439


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1013,3 +1027,359 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+BOUND_TRUNCATED_LENGHT = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+logger = logging.getLogger(__name__)
+
+# Serialization rules: https://iceberg.apache.org/spec/#binary-single-value-serialization
+#
+# Type      Binary serialization
+# boolean   0x00 for false, non-zero byte for true
+# int       Stored as 4-byte little-endian
+# long      Stored as 8-byte little-endian
+# float     Stored as 4-byte little-endian
+# double    Stored as 8-byte little-endian
+# date      Stores days from the 1970-01-01 in an 4-byte little-endian int
+# time      Stores microseconds from midnight in an 8-byte little-endian long
+# timestamp without zone	Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long
+# timestamp with zone	Stores microseconds from 1970-01-01 00:00:00.000000 UTC in an 8-byte little-endian long
+# string    UTF-8 bytes (without length)
+# uuid      16-byte big-endian value, see example in Appendix B
+# fixed(L)  Binary value
+# binary    Binary value (without length)
+#
+
+
+def bool_to_avro(value: bool) -> bytes:
+    return STRUCT_BOOL.pack(value)
+
+
+def int32_to_avro(value: int) -> bytes:
+    return STRUCT_INT32.pack(value)
+
+
+def int64_to_avro(value: int) -> bytes:
+    return STRUCT_INT64.pack(value)
+
+
+def float_to_avro(value: float) -> bytes:
+    return STRUCT_FLOAT.pack(value)
+
+
+def double_to_avro(value: float) -> bytes:
+    return STRUCT_DOUBLE.pack(value)
+
+
+def bytes_to_avro(value: Union[bytes, str]) -> bytes:
+    if type(value) == str:
+        return value.encode()
+    else:
+        assert isinstance(value, bytes)  # appeases mypy
+        return value
+
+
+class StatsAggregator:
+    def __init__(self, type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.serialize: Any = None
+        self.trunc_lenght = trunc_length
+
+        if type_string == "BOOLEAN":
+            self.serialize = bool_to_avro
+        elif type_string == "INT32":
+            self.serialize = int32_to_avro
+        elif type_string == "INT64":
+            self.serialize = int64_to_avro
+        elif type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+        elif type_string == "FLOAT":
+            self.serialize = float_to_avro
+        elif type_string == "DOUBLE":
+            self.serialize = double_to_avro
+        elif type_string == "BYTE_ARRAY":
+            self.serialize = bytes_to_avro
+        elif type_string == "FIXED_LEN_BYTE_ARRAY":
+            self.serialize = bytes_to_avro
+        else:
+            raise AssertionError(f"Unknown physical type {type_string}")
+
+    def add_min(self, val: bytes) -> None:
+        if self.current_min is None:
+            self.current_min = val
+        else:
+            self.current_min = min(val, self.current_min)
+
+    def add_max(self, val: bytes) -> None:
+        if self.current_max is None:
+            self.current_max = val
+        else:
+            self.current_max = max(self.current_max, val)
+
+    def get_min(self) -> bytes:
+        return self.serialize(self.current_min)[: self.trunc_lenght]
+
+    def get_max(self) -> bytes:
+        return self.serialize(self.current_max)[: self.trunc_lenght]
+
+
+class MetricsMode(Enum):
+    NONE = 0
+    COUNTS = 1
+    TRUNC = 2

Review Comment:
   I think an Enum is a more pythonic way to define a set of constants.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] maxdebayser commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "maxdebayser (via GitHub)" <gi...@apache.org>.
maxdebayser commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1258896366


##########
python/pyiceberg/utils/file_stats.py:
##########
@@ -0,0 +1,333 @@
+#  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.
+
+import struct
+from typing import (
+    Any,
+    Dict,
+    List,
+    Union,
+)
+
+import pyarrow.lib
+import pyarrow.parquet as pq
+
+from pyiceberg.manifest import DataFile, FileFormat
+from pyiceberg.schema import Schema, SchemaVisitor, visit
+from pyiceberg.types import (
+    IcebergType,
+    ListType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StructType,
+)
+
+BOUND_TRUNCATED_LENGHT = 16
+
+# Serialization rules: https://iceberg.apache.org/spec/#binary-single-value-serialization
+#
+# Type      Binary serialization
+# boolean   0x00 for false, non-zero byte for true
+# int       Stored as 4-byte little-endian
+# long      Stored as 8-byte little-endian
+# float     Stored as 4-byte little-endian
+# double    Stored as 8-byte little-endian
+# date      Stores days from the 1970-01-01 in an 4-byte little-endian int
+# time      Stores microseconds from midnight in an 8-byte little-endian long
+# timestamp without zone	Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long
+# timestamp with zone	Stores microseconds from 1970-01-01 00:00:00.000000 UTC in an 8-byte little-endian long
+# string    UTF-8 bytes (without length)
+# uuid      16-byte big-endian value, see example in Appendix B
+# fixed(L)  Binary value
+# binary    Binary value (without length)
+#
+
+
+def bool_to_avro(value: bool) -> bytes:
+    return struct.pack("?", value)
+
+
+def int32_to_avro(value: int) -> bytes:
+    return struct.pack("<i", value)
+
+
+def int64_to_avro(value: int) -> bytes:
+    return struct.pack("<q", value)
+
+
+def float_to_avro(value: float) -> bytes:
+    return struct.pack("<f", value)
+
+
+def double_to_avro(value: float) -> bytes:
+    return struct.pack("<d", value)
+
+
+def bytes_to_avro(value: Union[bytes, str]) -> bytes:
+    if type(value) == str:
+        return value.encode()
+    else:
+        assert isinstance(value, bytes)  # appeases mypy
+        return value
+
+
+class StatsAggregator:
+    def __init__(self, type_string: str):
+        self.current_min: Any = None
+        self.current_max: Any = None
+        self.serialize: Any = None
+
+        if type_string == "BOOLEAN":
+            self.serialize = bool_to_avro
+        elif type_string == "INT32":
+            self.serialize = int32_to_avro
+        elif type_string == "INT64":
+            self.serialize = int64_to_avro
+        elif type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+        elif type_string == "FLOAT":
+            self.serialize = float_to_avro
+        elif type_string == "DOUBLE":
+            self.serialize = double_to_avro
+        elif type_string == "BYTE_ARRAY":
+            self.serialize = bytes_to_avro
+        elif type_string == "FIXED_LEN_BYTE_ARRAY":
+            self.serialize = bytes_to_avro
+        else:
+            raise AssertionError(f"Unknown physical type {type_string}")
+
+    def add_min(self, val: bytes) -> None:
+        if not self.current_min:
+            self.current_min = val
+        elif val < self.current_min:
+            self.current_min = val
+
+    def add_max(self, val: bytes) -> None:
+        if not self.current_max:
+            self.current_max = val
+        elif self.current_max < val:
+            self.current_max = val
+
+    def get_min(self) -> bytes:
+        return self.serialize(self.current_min)[:BOUND_TRUNCATED_LENGHT]
+
+    def get_max(self) -> bytes:
+        return self.serialize(self.current_max)[:BOUND_TRUNCATED_LENGHT]
+
+
+def fill_parquet_file_metadata(
+    df: DataFile, metadata: pq.FileMetaData, col_path_2_iceberg_id: Dict[str, int], file_size: int

Review Comment:
   Thanks for pointing that out. I've moved the code over.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] maxdebayser commented on pull request #7831: Compute parquet stats

Posted by "maxdebayser (via GitHub)" <gi...@apache.org>.
maxdebayser commented on PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#issuecomment-1608418483

   @Fokko , I've rewritten the implementation to use the pyarrow metadata_collector.  I've added a test cases to make sure that it's compatible with `write_dataset` as well as `ParquetWriter`. I've also also added a function to map the path of columns in the parquet metadata to Iceberg field IDs.


-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1285260812


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1039,335 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.values()).union({"INT96"})
+
+
+class StatsAggregator:
+    current_min: Any
+    current_max: Any
+    trunc_length: Optional[int]
+
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min = None
+        self.current_max = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        expected_physical_type = _PRIMITIVE_TO_PHYSICAL[iceberg_type]
+        if expected_physical_type != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {expected_physical_type}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        self.current_min = val if self.current_min is None else min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        self.current_max = val if self.current_max is None else max(val, self.current_max)
+
+    def get_min(self) -> bytes:
+        return self.serialize(
+            self.current_min
+            if self.trunc_length is None
+            else TruncateTransform(width=self.trunc_length).transform(self.primitive_type)(self.current_min)
+        )
+
+    def get_max(self) -> Optional[bytes]:
+        if self.current_max is None:
+            return None
+
+        if self.primitive_type == StringType():
+            if type(self.current_max) != str:
+                raise ValueError("Expected the current_max to be a string")
+
+            s_result = self.current_max[: self.trunc_length]
+            if s_result != self.current_max:
+                chars = [*s_result]
+
+                for i in range(-1, -len(s_result) - 1, -1):
+                    try:
+                        to_inc = ord(chars[i])
+                        # will raise exception if the highest unicode code is reached
+                        _next = chr(to_inc + 1)
+                        chars[i] = _next
+                        return self.serialize("".join(chars))
+                    except ValueError:
+                        pass
+                return None  # didn't find a valid upper bound
+            return self.serialize(s_result)
+        elif self.primitive_type == BinaryType():
+            if type(self.current_max) != bytes:
+                raise ValueError("Expected the current_max to be bytes")
+            b_result = self.current_max[: self.trunc_length]
+            if b_result != self.current_max:
+                _bytes = [*b_result]
+                for i in range(-1, -len(b_result) - 1, -1):
+                    if _bytes[i] < 255:
+                        _bytes[i] += 1
+                        return b"".join([i.to_bytes(1, byteorder="little") for i in _bytes])
+                return None
+
+            return self.serialize(b_result)
+        else:
+            return self.serialize(self.current_max)[: self.trunc_length]
+
+
+DEFAULT_TRUNCATION_LENGTH = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+
+class MetricModeTypes(Enum):
+    TRUNCATE = "truncate"
+    NONE = "none"
+    COUNTS = "counts"
+    FULL = "full"
+
+
+DEFAULT_METRICS_MODE_KEY = "write.metadata.metrics.default"
+COLUMN_METRICS_MODE_KEY_PREFIX = "write.metadata.metrics.column"
+
+
+@dataclass(frozen=True)
+class MetricsMode(Singleton):
+    type: MetricModeTypes
+    length: Optional[int] = None
+
+
+def match_metrics_mode(mode: str) -> MetricsMode:
+    sanitized_mode = mode.lower()
+    if sanitized_mode.startswith("truncate"):
+        m = re.match(TRUNCATION_EXPR, mode, re.IGNORECASE)
+        if m:
+            length = int(m[1])
+            if length < 1:
+                raise ValueError("Truncation length must be larger than 0")
+            return MetricsMode(MetricModeTypes.TRUNCATE, int(m[1]))
+        else:
+            raise ValueError(f"Malformed truncate: {mode}")
+    elif sanitized_mode.startswith("none"):
+        return MetricsMode(MetricModeTypes.NONE)
+    elif sanitized_mode.startswith("counts"):
+        return MetricsMode(MetricModeTypes.COUNTS)
+    elif sanitized_mode.startswith("full"):
+        return MetricsMode(MetricModeTypes.FULL)
+    else:
+        raise ValueError(f"Unsupported metrics mode: {mode}")
+
+
+@dataclass(frozen=True)
+class StatisticsCollector:
+    field_id: int
+    iceberg_type: PrimitiveType
+    mode: MetricsMode
+    column_name: str
+
+
+class PyArrowStatisticsCollector(PreOrderSchemaVisitor[List[StatisticsCollector]]):
+    _field_id: int = 0
+    _schema: Schema
+    _properties: Dict[str, str]
+
+    def __init__(self, schema: Schema, properties: Dict[str, str]):
+        self._schema = schema
+        self._properties = properties
+
+    def schema(self, schema: Schema, struct_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        return struct_result()
+
+    def struct(
+        self, struct: StructType, field_results: List[Callable[[], List[StatisticsCollector]]]
+    ) -> List[StatisticsCollector]:
+        return list(chain(*[result() for result in field_results]))
+
+    def field(self, field: NestedField, field_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = field.field_id
+        result = field_result()
+        return result
+
+    def list(self, list_type: ListType, element_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = list_type.element_id
+        return element_result()
+
+    def map(
+        self,
+        map_type: MapType,
+        key_result: Callable[[], List[StatisticsCollector]],
+        value_result: Callable[[], List[StatisticsCollector]],
+    ) -> List[StatisticsCollector]:
+        self._field_id = map_type.key_id
+        k = key_result()
+        self._field_id = map_type.value_id
+        v = value_result()
+        return k + v
+
+    def primitive(self, primitive: PrimitiveType) -> List[StatisticsCollector]:
+        column_name = self._schema.find_column_name(self._field_id)
+        if column_name is None:
+            raise ValueError(f"Column for field {self._field_id} not found")
+
+        metrics_mode = MetricsMode(MetricModeTypes.TRUNCATE, DEFAULT_TRUNCATION_LENGTH)
+
+        default_mode = self._properties.get(DEFAULT_METRICS_MODE_KEY)
+        if default_mode:
+            metrics_mode = match_metrics_mode(default_mode)
+
+        col_mode = self._properties.get(f"{COLUMN_METRICS_MODE_KEY_PREFIX}.{column_name}")
+        if col_mode:
+            metrics_mode = match_metrics_mode(col_mode)
+
+        if (
+            not (isinstance(primitive, StringType) or isinstance(primitive, BinaryType))
+            and metrics_mode.type == MetricModeTypes.TRUNCATE
+        ):
+            metrics_mode = MetricsMode(MetricModeTypes.FULL)
+
+        return [StatisticsCollector(field_id=self._field_id, iceberg_type=primitive, mode=metrics_mode, column_name=column_name)]
+
+
+def fill_parquet_file_metadata(
+    df: DataFile,
+    parquet_metadata: pq.FileMetaData,
+    file_size: int,
+    table_metadata: TableMetadata,
+) -> None:
+    """
+    Computes and fills the following fields of the DataFile object.
+
+    - file_format
+    - record_count
+    - file_size_in_bytes
+    - column_sizes
+    - value_counts
+    - null_value_counts
+    - nan_value_counts
+    - lower_bounds
+    - upper_bounds
+    - split_offsets
+
+    Args:
+        df (DataFile): A DataFile object representing the Parquet file for which metadata is to be filled.
+        parquet_metadata (pyarrow.parquet.FileMetaData): A pyarrow metadata object.
+        file_size (int): The total compressed file size cannot be retrieved from the metadata and hence has to
+            be passed here. Depending on the kind of file system and pyarrow library call used, different
+            ways to obtain this value might be appropriate.
+        table_metadata (pyiceberg.table.metadata.TableMetadata): The Iceberg table metadata. It is required to
+            compute the mapping if column position to iceberg schema type id. It's also used to set the mode
+            for column metrics collection
+    """
+    schema = next(filter(lambda s: s.schema_id == table_metadata.current_schema_id, table_metadata.schemas))
+
+    stats_columns = pre_order_visit(schema, PyArrowStatisticsCollector(schema, table_metadata.properties))
+
+    if parquet_metadata.num_columns != len(stats_columns):
+        raise ValueError(
+            f"Number of columns in metadata ({len(stats_columns)}) is different from the number of columns in pyarrow table ({parquet_metadata.num_columns})"
+        )
+
+    column_sizes: Dict[int, int] = {}
+    value_counts: Dict[int, int] = {}
+    split_offsets: List[int] = []
+
+    null_value_counts: Dict[int, int] = {}
+    nan_value_counts: Dict[int, int] = {}
+
+    col_aggs = {}
+
+    for r in range(parquet_metadata.num_row_groups):
+        # References:
+        # https://github.com/apache/iceberg/blob/fc381a81a1fdb8f51a0637ca27cd30673bd7aad3/parquet/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java#L232
+        # https://github.com/apache/parquet-mr/blob/ac29db4611f86a07cc6877b416aa4b183e09b353/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/ColumnChunkMetaData.java#L184
+
+        row_group = parquet_metadata.row_group(r)
+
+        data_offset = row_group.column(0).data_page_offset
+        dictionary_offset = row_group.column(0).dictionary_page_offset
+
+        if row_group.column(0).has_dictionary_page and dictionary_offset < data_offset:
+            split_offsets.append(dictionary_offset)
+        else:
+            split_offsets.append(data_offset)
+
+        for pos, stats_col in enumerate(stats_columns):
+            field_id = stats_col.field_id
+
+            column = row_group.column(pos)
+
+            column_sizes[field_id] = column_sizes.get(field_id, 0) + column.total_compressed_size

Review Comment:
   Minor: seems cleaner to me to use `setdefault`:
   
   ```python
   column_sizes.setdefault(field_id, 0)
   column_sizes[field_id] += column.total_compressed_size
   ```



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] maxdebayser commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "maxdebayser (via GitHub)" <gi...@apache.org>.
maxdebayser commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1285960010


##########
python/pyiceberg/utils/file_stats.py:
##########
@@ -0,0 +1,164 @@
+#  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.
+
+from pyiceberg.manifest import DataFile, FileFormat
+import pyarrow.parquet as pq
+import pyarrow.compute as pc
+import pyarrow as pa
+import struct
+import datetime
+
+BOUND_TRUNCATED_LENGHT = 16
+
+# Serialization rules: https://iceberg.apache.org/spec/#binary-single-value-serialization
+#
+# Type      Binary serialization
+# boolean   0x00 for false, non-zero byte for true
+# int       Stored as 4-byte little-endian
+# long      Stored as 8-byte little-endian
+# float     Stored as 4-byte little-endian
+# double    Stored as 8-byte little-endian
+# date      Stores days from the 1970-01-01 in an 4-byte little-endian int
+# time      Stores microseconds from midnight in an 8-byte little-endian long
+# timestamp without zone	Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long
+# timestamp with zone	Stores microseconds from 1970-01-01 00:00:00.000000 UTC in an 8-byte little-endian long
+# string    UTF-8 bytes (without length)
+# uuid      16-byte big-endian value, see example in Appendix B
+# fixed(L)  Binary value
+# binary    Binary value (without length)
+#
+def serialize_to_binary(scalar: pa.Scalar) -> bytes:
+    value = scalar.as_py()
+    if isinstance(scalar, pa.BooleanScalar):
+        return struct.pack('?', value)
+    elif isinstance(scalar, (pa.Int8Scalar, pa.UInt8Scalar)):
+        return struct.pack('<b', value)
+    elif isinstance(scalar, (pa.Int16Scalar, pa.UInt16Scalar)):
+        return struct.pack('<h', value)
+    elif isinstance(scalar, (pa.Int32Scalar, pa.UInt32Scalar)):
+        return struct.pack('<i', value)
+    elif isinstance(scalar, (pa.Int64Scalar, pa.UInt64Scalar)):
+        return struct.pack('<q', value)
+    elif isinstance(scalar, pa.FloatScalar):
+        return struct.pack('<f', value)
+    elif isinstance(scalar, pa.DoubleScalar):
+        return struct.pack('<d', value)
+    elif isinstance(scalar, pa.StringScalar):
+        return value.encode('utf-8')
+    elif isinstance(scalar, pa.BinaryScalar):
+        return value
+    elif isinstance(scalar, (pa.Date32Scalar, pa.Date64Scalar)):
+        epoch = datetime.date(1970, 1, 1)
+        days = (value - epoch).days
+        return struct.pack('<i', days)
+    elif isinstance(scalar, (pa.Time32Scalar, pa.Time64Scalar)):
+        microseconds = int(value.hour * 60 * 60 * 1e6 +
+                        value.minute * 60 * 1e6 +
+                        value.second * 1e6 +
+                        value.microsecond)
+        return struct.pack('<q', microseconds)
+    elif isinstance(scalar, pa.TimestampScalar):
+        epoch = datetime.datetime(1970, 1, 1)
+        microseconds = int((value - epoch).total_seconds() * 1e6)
+        return struct.pack('<q', microseconds)
+    else:
+        raise TypeError('Unsupported type: {}'.format(type(scalar)))
+
+
+def fill_parquet_file_metadata(df: DataFile, file_object: pa.NativeFile, table: pa.Table = None) -> None:
+    """
+    Computes and fills the following fields of the DataFile object:
+
+    - file_format
+    - record_count
+    - file_size_in_bytes
+    - column_sizes
+    - value_counts
+    - null_value_counts
+    - nan_value_counts
+    - lower_bounds
+    - upper_bounds
+    - split_offsets
+    
+    Args:
+        df (DataFile): A DataFile object representing the Parquet file for which metadata is to be filled.
+        file_object (pa.NativeFile): A pyarrow NativeFile object pointing to the location where the 
+            Parquet file is stored.
+        table (pa.Table, optional): If the metadata is computed while writing a pyarrow Table to parquet
+            the table can be passed to compute the column statistics. If absent the table will be read
+            from file_object using pyarrow.parquet.read_table.
+    """
+    
+    parquet_file = pq.ParquetFile(file_object)

Review Comment:
   Yes, collecting the statistics while writing has been implemented.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] maxdebayser commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "maxdebayser (via GitHub)" <gi...@apache.org>.
maxdebayser commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1285964561


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1039,335 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.values()).union({"INT96"})
+
+
+class StatsAggregator:
+    current_min: Any
+    current_max: Any
+    trunc_length: Optional[int]
+
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min = None
+        self.current_max = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        expected_physical_type = _PRIMITIVE_TO_PHYSICAL[iceberg_type]
+        if expected_physical_type != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {expected_physical_type}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        self.current_min = val if self.current_min is None else min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        self.current_max = val if self.current_max is None else max(val, self.current_max)
+
+    def get_min(self) -> bytes:
+        return self.serialize(
+            self.current_min
+            if self.trunc_length is None
+            else TruncateTransform(width=self.trunc_length).transform(self.primitive_type)(self.current_min)

Review Comment:
   Yes, it's called only once.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] maxdebayser commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "maxdebayser (via GitHub)" <gi...@apache.org>.
maxdebayser commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1286009490


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1039,335 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.values()).union({"INT96"})
+
+
+class StatsAggregator:
+    current_min: Any
+    current_max: Any
+    trunc_length: Optional[int]
+
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min = None
+        self.current_max = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        expected_physical_type = _PRIMITIVE_TO_PHYSICAL[iceberg_type]
+        if expected_physical_type != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {expected_physical_type}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        self.current_min = val if self.current_min is None else min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        self.current_max = val if self.current_max is None else max(val, self.current_max)
+
+    def get_min(self) -> bytes:
+        return self.serialize(
+            self.current_min
+            if self.trunc_length is None
+            else TruncateTransform(width=self.trunc_length).transform(self.primitive_type)(self.current_min)
+        )
+
+    def get_max(self) -> Optional[bytes]:
+        if self.current_max is None:
+            return None
+
+        if self.primitive_type == StringType():
+            if type(self.current_max) != str:
+                raise ValueError("Expected the current_max to be a string")
+
+            s_result = self.current_max[: self.trunc_length]
+            if s_result != self.current_max:
+                chars = [*s_result]
+
+                for i in range(-1, -len(s_result) - 1, -1):
+                    try:
+                        to_inc = ord(chars[i])
+                        # will raise exception if the highest unicode code is reached
+                        _next = chr(to_inc + 1)
+                        chars[i] = _next
+                        return self.serialize("".join(chars))
+                    except ValueError:
+                        pass
+                return None  # didn't find a valid upper bound
+            return self.serialize(s_result)
+        elif self.primitive_type == BinaryType():
+            if type(self.current_max) != bytes:
+                raise ValueError("Expected the current_max to be bytes")
+            b_result = self.current_max[: self.trunc_length]
+            if b_result != self.current_max:
+                _bytes = [*b_result]
+                for i in range(-1, -len(b_result) - 1, -1):
+                    if _bytes[i] < 255:
+                        _bytes[i] += 1
+                        return b"".join([i.to_bytes(1, byteorder="little") for i in _bytes])
+                return None
+
+            return self.serialize(b_result)
+        else:
+            return self.serialize(self.current_max)[: self.trunc_length]
+
+
+DEFAULT_TRUNCATION_LENGTH = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+
+class MetricModeTypes(Enum):
+    TRUNCATE = "truncate"
+    NONE = "none"
+    COUNTS = "counts"
+    FULL = "full"
+
+
+DEFAULT_METRICS_MODE_KEY = "write.metadata.metrics.default"
+COLUMN_METRICS_MODE_KEY_PREFIX = "write.metadata.metrics.column"
+
+
+@dataclass(frozen=True)
+class MetricsMode(Singleton):
+    type: MetricModeTypes
+    length: Optional[int] = None
+
+
+def match_metrics_mode(mode: str) -> MetricsMode:
+    sanitized_mode = mode.lower()
+    if sanitized_mode.startswith("truncate"):
+        m = re.match(TRUNCATION_EXPR, mode, re.IGNORECASE)
+        if m:
+            length = int(m[1])
+            if length < 1:
+                raise ValueError("Truncation length must be larger than 0")
+            return MetricsMode(MetricModeTypes.TRUNCATE, int(m[1]))
+        else:
+            raise ValueError(f"Malformed truncate: {mode}")
+    elif sanitized_mode.startswith("none"):

Review Comment:
   It didn't, it's an artifact of the review process. The first version of the code matched beginning and end of the string:
   ```
   def match_metrics_mode(mode: str) -> MetricsMode:
       m = re.match(TRUNCATION_EXPR, mode, re.IGNORECASE)
       if m:
           length = int(m[1])
           if length < 1:
               raise ValueError("Truncation length must be larger than 0")
           return MetricsMode(MetricModeTypes.TRUNCATE, int(m[1]))
       elif re.match("^none$", mode, re.IGNORECASE):
           return MetricsMode(MetricModeTypes.NONE)
       elif re.match("^counts$", mode, re.IGNORECASE):
           return MetricsMode(MetricModeTypes.COUNTS)
       elif re.match("^full$", mode, re.IGNORECASE):
           return MetricsMode(MetricModeTypes.FULL)
       else:
           raise ValueError(f"Unsupported metrics mode {mode}")
   ```



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1285261136


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1039,335 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.values()).union({"INT96"})
+
+
+class StatsAggregator:
+    current_min: Any
+    current_max: Any
+    trunc_length: Optional[int]
+
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min = None
+        self.current_max = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        expected_physical_type = _PRIMITIVE_TO_PHYSICAL[iceberg_type]
+        if expected_physical_type != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {expected_physical_type}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)
+
+        return to_bytes(self.primitive_type, value)
+
+    def add_min(self, val: Any) -> None:
+        self.current_min = val if self.current_min is None else min(val, self.current_min)
+
+    def add_max(self, val: Any) -> None:
+        self.current_max = val if self.current_max is None else max(val, self.current_max)
+
+    def get_min(self) -> bytes:
+        return self.serialize(
+            self.current_min
+            if self.trunc_length is None
+            else TruncateTransform(width=self.trunc_length).transform(self.primitive_type)(self.current_min)
+        )
+
+    def get_max(self) -> Optional[bytes]:
+        if self.current_max is None:
+            return None
+
+        if self.primitive_type == StringType():
+            if type(self.current_max) != str:
+                raise ValueError("Expected the current_max to be a string")
+
+            s_result = self.current_max[: self.trunc_length]
+            if s_result != self.current_max:
+                chars = [*s_result]
+
+                for i in range(-1, -len(s_result) - 1, -1):
+                    try:
+                        to_inc = ord(chars[i])
+                        # will raise exception if the highest unicode code is reached
+                        _next = chr(to_inc + 1)
+                        chars[i] = _next
+                        return self.serialize("".join(chars))
+                    except ValueError:
+                        pass
+                return None  # didn't find a valid upper bound
+            return self.serialize(s_result)
+        elif self.primitive_type == BinaryType():
+            if type(self.current_max) != bytes:
+                raise ValueError("Expected the current_max to be bytes")
+            b_result = self.current_max[: self.trunc_length]
+            if b_result != self.current_max:
+                _bytes = [*b_result]
+                for i in range(-1, -len(b_result) - 1, -1):
+                    if _bytes[i] < 255:
+                        _bytes[i] += 1
+                        return b"".join([i.to_bytes(1, byteorder="little") for i in _bytes])
+                return None
+
+            return self.serialize(b_result)
+        else:
+            return self.serialize(self.current_max)[: self.trunc_length]
+
+
+DEFAULT_TRUNCATION_LENGTH = 16
+TRUNCATION_EXPR = r"^truncate\((\d+)\)$"
+
+
+class MetricModeTypes(Enum):
+    TRUNCATE = "truncate"
+    NONE = "none"
+    COUNTS = "counts"
+    FULL = "full"
+
+
+DEFAULT_METRICS_MODE_KEY = "write.metadata.metrics.default"
+COLUMN_METRICS_MODE_KEY_PREFIX = "write.metadata.metrics.column"
+
+
+@dataclass(frozen=True)
+class MetricsMode(Singleton):
+    type: MetricModeTypes
+    length: Optional[int] = None
+
+
+def match_metrics_mode(mode: str) -> MetricsMode:
+    sanitized_mode = mode.lower()
+    if sanitized_mode.startswith("truncate"):
+        m = re.match(TRUNCATION_EXPR, mode, re.IGNORECASE)
+        if m:
+            length = int(m[1])
+            if length < 1:
+                raise ValueError("Truncation length must be larger than 0")
+            return MetricsMode(MetricModeTypes.TRUNCATE, int(m[1]))
+        else:
+            raise ValueError(f"Malformed truncate: {mode}")
+    elif sanitized_mode.startswith("none"):
+        return MetricsMode(MetricModeTypes.NONE)
+    elif sanitized_mode.startswith("counts"):
+        return MetricsMode(MetricModeTypes.COUNTS)
+    elif sanitized_mode.startswith("full"):
+        return MetricsMode(MetricModeTypes.FULL)
+    else:
+        raise ValueError(f"Unsupported metrics mode: {mode}")
+
+
+@dataclass(frozen=True)
+class StatisticsCollector:
+    field_id: int
+    iceberg_type: PrimitiveType
+    mode: MetricsMode
+    column_name: str
+
+
+class PyArrowStatisticsCollector(PreOrderSchemaVisitor[List[StatisticsCollector]]):
+    _field_id: int = 0
+    _schema: Schema
+    _properties: Dict[str, str]
+
+    def __init__(self, schema: Schema, properties: Dict[str, str]):
+        self._schema = schema
+        self._properties = properties
+
+    def schema(self, schema: Schema, struct_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        return struct_result()
+
+    def struct(
+        self, struct: StructType, field_results: List[Callable[[], List[StatisticsCollector]]]
+    ) -> List[StatisticsCollector]:
+        return list(chain(*[result() for result in field_results]))
+
+    def field(self, field: NestedField, field_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = field.field_id
+        result = field_result()
+        return result
+
+    def list(self, list_type: ListType, element_result: Callable[[], List[StatisticsCollector]]) -> List[StatisticsCollector]:
+        self._field_id = list_type.element_id
+        return element_result()
+
+    def map(
+        self,
+        map_type: MapType,
+        key_result: Callable[[], List[StatisticsCollector]],
+        value_result: Callable[[], List[StatisticsCollector]],
+    ) -> List[StatisticsCollector]:
+        self._field_id = map_type.key_id
+        k = key_result()
+        self._field_id = map_type.value_id
+        v = value_result()
+        return k + v
+
+    def primitive(self, primitive: PrimitiveType) -> List[StatisticsCollector]:
+        column_name = self._schema.find_column_name(self._field_id)
+        if column_name is None:
+            raise ValueError(f"Column for field {self._field_id} not found")
+
+        metrics_mode = MetricsMode(MetricModeTypes.TRUNCATE, DEFAULT_TRUNCATION_LENGTH)
+
+        default_mode = self._properties.get(DEFAULT_METRICS_MODE_KEY)
+        if default_mode:
+            metrics_mode = match_metrics_mode(default_mode)
+
+        col_mode = self._properties.get(f"{COLUMN_METRICS_MODE_KEY_PREFIX}.{column_name}")
+        if col_mode:
+            metrics_mode = match_metrics_mode(col_mode)
+
+        if (
+            not (isinstance(primitive, StringType) or isinstance(primitive, BinaryType))
+            and metrics_mode.type == MetricModeTypes.TRUNCATE
+        ):
+            metrics_mode = MetricsMode(MetricModeTypes.FULL)
+
+        return [StatisticsCollector(field_id=self._field_id, iceberg_type=primitive, mode=metrics_mode, column_name=column_name)]
+
+
+def fill_parquet_file_metadata(
+    df: DataFile,
+    parquet_metadata: pq.FileMetaData,
+    file_size: int,
+    table_metadata: TableMetadata,
+) -> None:
+    """
+    Computes and fills the following fields of the DataFile object.
+
+    - file_format
+    - record_count
+    - file_size_in_bytes
+    - column_sizes
+    - value_counts
+    - null_value_counts
+    - nan_value_counts
+    - lower_bounds
+    - upper_bounds
+    - split_offsets
+
+    Args:
+        df (DataFile): A DataFile object representing the Parquet file for which metadata is to be filled.
+        parquet_metadata (pyarrow.parquet.FileMetaData): A pyarrow metadata object.
+        file_size (int): The total compressed file size cannot be retrieved from the metadata and hence has to
+            be passed here. Depending on the kind of file system and pyarrow library call used, different
+            ways to obtain this value might be appropriate.
+        table_metadata (pyiceberg.table.metadata.TableMetadata): The Iceberg table metadata. It is required to
+            compute the mapping if column position to iceberg schema type id. It's also used to set the mode
+            for column metrics collection
+    """
+    schema = next(filter(lambda s: s.schema_id == table_metadata.current_schema_id, table_metadata.schemas))
+
+    stats_columns = pre_order_visit(schema, PyArrowStatisticsCollector(schema, table_metadata.properties))
+
+    if parquet_metadata.num_columns != len(stats_columns):
+        raise ValueError(
+            f"Number of columns in metadata ({len(stats_columns)}) is different from the number of columns in pyarrow table ({parquet_metadata.num_columns})"
+        )
+
+    column_sizes: Dict[int, int] = {}
+    value_counts: Dict[int, int] = {}
+    split_offsets: List[int] = []
+
+    null_value_counts: Dict[int, int] = {}
+    nan_value_counts: Dict[int, int] = {}
+
+    col_aggs = {}
+
+    for r in range(parquet_metadata.num_row_groups):
+        # References:
+        # https://github.com/apache/iceberg/blob/fc381a81a1fdb8f51a0637ca27cd30673bd7aad3/parquet/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java#L232
+        # https://github.com/apache/parquet-mr/blob/ac29db4611f86a07cc6877b416aa4b183e09b353/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/ColumnChunkMetaData.java#L184
+
+        row_group = parquet_metadata.row_group(r)
+
+        data_offset = row_group.column(0).data_page_offset
+        dictionary_offset = row_group.column(0).dictionary_page_offset
+
+        if row_group.column(0).has_dictionary_page and dictionary_offset < data_offset:
+            split_offsets.append(dictionary_offset)
+        else:
+            split_offsets.append(data_offset)
+
+        for pos, stats_col in enumerate(stats_columns):
+            field_id = stats_col.field_id
+
+            column = row_group.column(pos)
+
+            column_sizes[field_id] = column_sizes.get(field_id, 0) + column.total_compressed_size
+
+            if stats_col.mode == MetricsMode(MetricModeTypes.NONE):
+                continue
+
+            value_counts[field_id] = value_counts.get(field_id, 0) + column.num_values
+
+            if column.is_stats_set:
+                try:
+                    statistics = column.statistics
+
+                    null_value_counts[field_id] = null_value_counts.get(field_id, 0) + statistics.null_count
+
+                    if stats_col.mode == MetricsMode(MetricModeTypes.COUNTS):
+                        continue
+
+                    if field_id not in col_aggs:
+                        col_aggs[field_id] = StatsAggregator(
+                            stats_col.iceberg_type, statistics.physical_type, stats_col.mode.length
+                        )
+
+                    col_aggs[field_id].add_min(statistics.min)
+                    col_aggs[field_id].add_max(statistics.max)
+
+                except pyarrow.lib.ArrowNotImplementedError as e:
+                    logger.warning(e)
+            else:
+                logger.warning("PyArrow statistics missing for column %d when writing file", pos)

Review Comment:
   If this (or an exception above) happens, then this needs to ensure that the lower and upper bounds are discarded if they are present for other row groups. In Java, this is done by [tracking field IDs](https://github.com/apache/iceberg/blob/master/parquet/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java#L137) for which at least one row group didn't have stats, then suppressing min/max later on.
   
   Otherwise, you could be missing stats for a row group and not know what the actual min/max is, but still produce lower and upper bounds based on the data that was actually there.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1285262467


##########
python/tests/io/test_pyarrow.py:
##########
@@ -1345,3 +1374,655 @@ def test_pyarrow_wrap_fsspec(example_task: FileScanTask, table_schema_simple: Sc
 bar: [[1,2,3]]
 baz: [[true,false,null]]"""
     )
+
+
+def construct_test_table() -> Tuple[Any, Any, Union[TableMetadataV1, TableMetadataV2]]:
+    table_metadata = {
+        "format-version": 2,
+        "location": "s3://bucket/test/location",
+        "last-column-id": 7,
+        "current-schema-id": 0,
+        "schemas": [
+            {
+                "type": "struct",
+                "schema-id": 0,
+                "fields": [
+                    {"id": 1, "name": "strings", "required": False, "type": "string"},
+                    {"id": 2, "name": "floats", "required": False, "type": "float"},
+                    {
+                        "id": 3,
+                        "name": "list",
+                        "required": False,
+                        "type": {"type": "list", "element-id": 5, "element": "long", "element-required": False},
+                    },
+                    {
+                        "id": 4,
+                        "name": "maps",
+                        "required": False,
+                        "type": {
+                            "type": "map",
+                            "key-id": 6,
+                            "key": "long",
+                            "value-id": 7,
+                            "value": "long",
+                            "value-required": False,
+                        },
+                    },
+                ],
+            },
+        ],
+        "default-spec-id": 0,
+        "partition-specs": [{"spec-id": 0, "fields": []}],
+        "properties": {},
+    }
+
+    table_metadata = TableMetadataUtil.parse_obj(table_metadata)
+    arrow_schema = schema_to_pyarrow(table_metadata.schemas[0])
+
+    _strings = ["zzzzzzzzzzzzzzzzzzzz", "rrrrrrrrrrrrrrrrrrrr", None, "aaaaaaaaaaaaaaaaaaaa"]
+
+    _floats = [3.14, math.nan, 1.69, 100]
+
+    _list = [[1, 2, 3], [4, 5, 6], None, [7, 8, 9]]
+
+    _maps: List[Optional[Dict[int, int]]] = [
+        {1: 2, 3: 4},
+        None,
+        {5: 6},
+        {},
+    ]
+
+    table = pa.Table.from_pydict(
+        {
+            "strings": _strings,
+            "floats": _floats,
+            "list": _list,
+            "maps": _maps,
+        },
+        schema=arrow_schema,
+    )
+    metadata_collector: List[Any] = []
+
+    with pa.BufferOutputStream() as f:
+        with pq.ParquetWriter(f, table.schema, metadata_collector=metadata_collector) as writer:
+            writer.write_table(table)
+
+        return f.getvalue(), metadata_collector[0], table_metadata
+
+
+def test_record_count() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.record_count == 4
+
+
+def test_file_size() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.file_size_in_bytes == len(file_bytes)
+
+
+def test_value_counts() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.value_counts) == 5
+    assert datafile.value_counts[1] == 4
+    assert datafile.value_counts[2] == 4
+    assert datafile.value_counts[5] == 10  # 3 lists with 3 items and a None value
+    assert datafile.value_counts[6] == 5
+    assert datafile.value_counts[7] == 5
+
+
+def test_column_sizes() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.column_sizes) == 5
+    # these values are an artifact of how the write_table encodes the columns

Review Comment:
   Do we want to test these directly or just assert that the value is > 0?



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1285261517


##########
python/tests/io/test_pyarrow.py:
##########
@@ -16,16 +16,39 @@
 # under the License.
 # pylint: disable=protected-access,unused-argument,redefined-outer-name
 
+import math
 import os
 import tempfile
-from typing import Any, List, Optional
+import uuid
+from datetime import (

Review Comment:
   @Fokko, why are there `datetime` classes here?
   
   I think that tests like these should completely avoid Python datetime logic and instead should directly use the internal 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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1288426617


##########
python/tests/io/test_pyarrow.py:
##########
@@ -1345,3 +1374,655 @@ def test_pyarrow_wrap_fsspec(example_task: FileScanTask, table_schema_simple: Sc
 bar: [[1,2,3]]
 baz: [[true,false,null]]"""
     )
+
+
+def construct_test_table() -> Tuple[Any, Any, Union[TableMetadataV1, TableMetadataV2]]:
+    table_metadata = {
+        "format-version": 2,
+        "location": "s3://bucket/test/location",
+        "last-column-id": 7,
+        "current-schema-id": 0,
+        "schemas": [
+            {
+                "type": "struct",
+                "schema-id": 0,
+                "fields": [
+                    {"id": 1, "name": "strings", "required": False, "type": "string"},
+                    {"id": 2, "name": "floats", "required": False, "type": "float"},
+                    {
+                        "id": 3,
+                        "name": "list",
+                        "required": False,
+                        "type": {"type": "list", "element-id": 5, "element": "long", "element-required": False},
+                    },
+                    {
+                        "id": 4,
+                        "name": "maps",
+                        "required": False,
+                        "type": {
+                            "type": "map",
+                            "key-id": 6,
+                            "key": "long",
+                            "value-id": 7,
+                            "value": "long",
+                            "value-required": False,
+                        },
+                    },
+                ],
+            },
+        ],
+        "default-spec-id": 0,
+        "partition-specs": [{"spec-id": 0, "fields": []}],
+        "properties": {},
+    }
+
+    table_metadata = TableMetadataUtil.parse_obj(table_metadata)
+    arrow_schema = schema_to_pyarrow(table_metadata.schemas[0])
+
+    _strings = ["zzzzzzzzzzzzzzzzzzzz", "rrrrrrrrrrrrrrrrrrrr", None, "aaaaaaaaaaaaaaaaaaaa"]
+
+    _floats = [3.14, math.nan, 1.69, 100]
+
+    _list = [[1, 2, 3], [4, 5, 6], None, [7, 8, 9]]
+
+    _maps: List[Optional[Dict[int, int]]] = [
+        {1: 2, 3: 4},
+        None,
+        {5: 6},
+        {},
+    ]
+
+    table = pa.Table.from_pydict(
+        {
+            "strings": _strings,
+            "floats": _floats,
+            "list": _list,
+            "maps": _maps,
+        },
+        schema=arrow_schema,
+    )
+    metadata_collector: List[Any] = []
+
+    with pa.BufferOutputStream() as f:
+        with pq.ParquetWriter(f, table.schema, metadata_collector=metadata_collector) as writer:
+            writer.write_table(table)
+
+        return f.getvalue(), metadata_collector[0], table_metadata
+
+
+def test_record_count() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.record_count == 4
+
+
+def test_file_size() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert datafile.file_size_in_bytes == len(file_bytes)
+
+
+def test_value_counts() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.value_counts) == 5
+    assert datafile.value_counts[1] == 4
+    assert datafile.value_counts[2] == 4
+    assert datafile.value_counts[5] == 10  # 3 lists with 3 items and a None value
+    assert datafile.value_counts[6] == 5
+    assert datafile.value_counts[7] == 5
+
+
+def test_column_sizes() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.column_sizes) == 5
+    # these values are an artifact of how the write_table encodes the columns
+    assert datafile.column_sizes[1] == 116
+    assert datafile.column_sizes[2] == 89
+    assert datafile.column_sizes[5] == 151
+    assert datafile.column_sizes[6] == 117
+    assert datafile.column_sizes[7] == 117
+
+
+def test_null_and_nan_counts() -> None:
+    (file_bytes, metadata, table_metadata) = construct_test_table()
+
+    datafile = DataFile()
+    fill_parquet_file_metadata(datafile, metadata, len(file_bytes), table_metadata)
+
+    assert len(datafile.null_value_counts) == 5
+    assert datafile.null_value_counts[1] == 1
+    assert datafile.null_value_counts[2] == 0
+    assert datafile.null_value_counts[5] == 1

Review Comment:
   🤔  Type:
   ```json
   {"type": "list", "element-id": 5, "element": "long", "element-required": False}
   ```
   And the data:
   ```python
   _list = [[1, 2, 3], [4, 5, 6], None, [7, 8, 9]]
   ```
   I count a single `null`



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on a diff in pull request #7831: Python: Compute parquet stats

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on code in PR #7831:
URL: https://github.com/apache/iceberg/pull/7831#discussion_r1289752108


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -1025,3 +1039,335 @@ def map_key_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]
 
     def map_value_partner(self, partner_map: Optional[pa.Array]) -> Optional[pa.Array]:
         return partner_map.items if isinstance(partner_map, pa.MapArray) else None
+
+
+_PRIMITIVE_TO_PHYSICAL = {
+    BooleanType(): "BOOLEAN",
+    IntegerType(): "INT32",
+    LongType(): "INT64",
+    FloatType(): "FLOAT",
+    DoubleType(): "DOUBLE",
+    DateType(): "INT32",
+    TimeType(): "INT64",
+    TimestampType(): "INT64",
+    TimestamptzType(): "INT64",
+    StringType(): "BYTE_ARRAY",
+    UUIDType(): "FIXED_LEN_BYTE_ARRAY",
+    BinaryType(): "BYTE_ARRAY",
+}
+_PHYSICAL_TYPES = set(_PRIMITIVE_TO_PHYSICAL.values()).union({"INT96"})
+
+
+class StatsAggregator:
+    current_min: Any
+    current_max: Any
+    trunc_length: Optional[int]
+
+    def __init__(self, iceberg_type: PrimitiveType, physical_type_string: str, trunc_length: Optional[int] = None) -> None:
+        self.current_min = None
+        self.current_max = None
+        self.trunc_length = trunc_length
+
+        if physical_type_string not in _PHYSICAL_TYPES:
+            raise ValueError(f"Unknown physical type {physical_type_string}")
+
+        if physical_type_string == "INT96":
+            raise NotImplementedError("Statistics not implemented for INT96 physical type")
+
+        expected_physical_type = _PRIMITIVE_TO_PHYSICAL[iceberg_type]
+        if expected_physical_type != physical_type_string:
+            raise ValueError(
+                f"Unexpected physical type {physical_type_string} for {iceberg_type}, expected {expected_physical_type}"
+            )
+
+        self.primitive_type = iceberg_type
+
+    def serialize(self, value: Any) -> bytes:
+        if self.primitive_type == UUIDType():
+            value = uuid.UUID(bytes=value)

Review Comment:
   @maxdebayser The PR is in, can you remove this check?
   
   We can now replace serialize in the constructor with:
   ```python
   from functools import partial
   self.serialize: Callable[[Any], bytes] = partial(to_bytes, self.primitive_type)
   ```
    



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org