You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by "JonasJ-ap (via GitHub)" <gi...@apache.org> on 2023/03/03 05:28:32 UTC

[GitHub] [iceberg] JonasJ-ap opened a new pull request, #6997: Python: Infer Iceberg schema from the Parquet file

JonasJ-ap opened a new pull request, #6997:
URL: https://github.com/apache/iceberg/pull/6997

   ## Problem Addressed:
   This PR fixes #6505, #6647 .
   It adds support to infer iceberg schema from parquet schema when the parquet file does not contain metadata holding the encoded iceberg schema.
   
   This PR is still under development. I put it here to receive some initial feedback about the overall design and proper structure. Thank you in advance for your help.
   
   ## Tests:
   Working on unit tests.
   
   Sample on AWS Athena (reproduce and fix the bug following the procedures in #6505 ):
   1. Create a table on AWS Glue:
   ```scala
   val type_frame = spark
                           .range(0, 5, 1, 5)
                           .withColumnRenamed("id", "longCol")
                           .withColumn("intCol", expr("CAST(longCol AS INT)"))
                           .withColumn("floatCol", expr("CAST(longCol AS FLOAT)"))
                           .withColumn("doubleCol", expr("CAST(longCol AS DOUBLE)"))
                           .withColumn("dateCol", date_add(current_date(), 1))
                           .withColumn("timestampCol", expr("TO_TIMESTAMP(dateCol)"))
                           .withColumn("stringCol", expr("CAST(dateCol AS STRING)"))
                           .withColumn("booleanCol", expr("longCol > 5"))
                           .withColumn("binaryCol", expr("CAST(longCol AS BINARY)"))
                           .withColumn("byteCol", expr("CAST(longCol AS BYTE)"))
                           .withColumn("decimalCol", expr("CAST(longCol AS DECIMAL(10, 2))"))
                           .withColumn("shortCol", expr("CAST(longCol AS SHORT)"))
                           .withColumn("mapCol", expr("MAP(longCol, decimalCol)"))
                           .withColumn("arrayCol", expr("ARRAY(longCol)"))
                           .withColumn("structCol", expr("STRUCT(mapCol, arrayCol)"))
   type_frame.writeTo(s"demo.$DB_NAME.type_test_ref_unpartitioned2").tableProperty("format-version", "2").createOrReplace()
   ```
   2. Use AWS Athena to optimize the table
   ```bash
   OPTIMIZE type_test_ref_unpartitioned REWRITE DATA USING BIN_PACK;
   ```
   3. Run code snippet leads to error:
   ```python
   from pyiceberg.catalog import load_catalog
   
   catalog = load_catalog("default", warehouse="s3://gluetestjonas/warehouse")
   table = catalog.load_table("iceberg_ref.type_test_ref_unpartitioned")
   
   df = table.scan().to_arrow()
   print(df)
   ```
   On current master branch:
   ```bash
   issue_6505 python3.10 parquet_schema.py           
   Traceback (most recent call last):
     File "/Users/jonasjiang/Workspace/Apache_Iceberg_ws/python_test/issue_6505/parquet_schema.py", line 6, in <module>
       df = table.scan().to_arrow()
     File "/Users/jonasjiang/Workspace/Apache_Iceberg_ws/iceberg_python/python/pyiceberg/table/__init__.py", line 404, in to_arrow
       return project_table(
     File "/Users/jonasjiang/Workspace/Apache_Iceberg_ws/iceberg_python/python/pyiceberg/io/pyarrow.py", line 777, in project_table
       for table in pool.starmap(
     File "/opt/homebrew/Cellar/python@3.10/3.10.10_1/Frameworks/Python.framework/Versions/3.10/lib/python3.10/multiprocessing/pool.py", line 375, in starmap
       return self._map_async(func, iterable, starmapstar, chunksize).get()
     File "/opt/homebrew/Cellar/python@3.10/3.10.10_1/Frameworks/Python.framework/Versions/3.10/lib/python3.10/multiprocessing/pool.py", line 774, in get
       raise self._value
     File "/opt/homebrew/Cellar/python@3.10/3.10.10_1/Frameworks/Python.framework/Versions/3.10/lib/python3.10/multiprocessing/pool.py", line 125, in worker
       result = (True, func(*args, **kwds))
     File "/opt/homebrew/Cellar/python@3.10/3.10.10_1/Frameworks/Python.framework/Versions/3.10/lib/python3.10/multiprocessing/pool.py", line 51, in starmapstar
       return list(itertools.starmap(args[0], args[1]))
     File "/Users/jonasjiang/Workspace/Apache_Iceberg_ws/iceberg_python/python/pyiceberg/io/pyarrow.py", line 714, in _file_to_table
       raise ValueError(
   ValueError: Iceberg schema is not embedded into the Parquet file, see https://github.com/apache/iceberg/issues/6505
   ```
   On this PR:
   ```bash
   issue_6505 python3.10 parquet_schema.py
   pyarrow.Table
   longCol: int64
   intCol: int32
   floatCol: float
   doubleCol: double
   dateCol: date32[day]
   timestampCol: timestamp[us, tz=UTC]
   stringCol: string
   booleanCol: bool
   binaryCol: binary
   byteCol: int32
   decimalCol: decimal128(10, 2)
   shortCol: int32
   mapCol: map<int64, decimal128(10, 2)>
     child 0, entries: struct<key: int64 not null, value: decimal128(10, 2)> not null
         child 0, key: int64 not null
         child 1, value: decimal128(10, 2)
   arrayCol: list<item: int64>
     child 0, item: int64
   structCol: struct<mapCol: map<int64, decimal128(10, 2)>, arrayCol: list<item: int64>>
     child 0, mapCol: map<int64, decimal128(10, 2)>
         child 0, entries: struct<key: int64 not null, value: decimal128(10, 2)> not null
             child 0, key: int64 not null
             child 1, value: decimal128(10, 2)
     child 1, arrayCol: list<item: int64>
         child 0, item: int64
   ----
   longCol: [[3,0,2,1,4]]
   intCol: [[3,0,2,1,4]]
   floatCol: [[3,0,2,1,4]]
   doubleCol: [[3,0,2,1,4]]
   dateCol: [[2023-03-04,2023-03-04,2023-03-04,2023-03-04,2023-03-04]]
   timestampCol: [[2023-03-04 00:00:00.000000,2023-03-04 00:00:00.000000,2023-03-04 00:00:00.000000,2023-03-04 00:00:00.000000,2023-03-04 00:00:00.000000]]
   stringCol: [["2023-03-04","2023-03-04","2023-03-04","2023-03-04","2023-03-04"]]
   booleanCol: [[false,false,false,false,false]]
   binaryCol: [[0000000000000003,0000000000000000,0000000000000002,0000000000000001,0000000000000004]]
   byteCol: [[3,0,2,1,4]]
   ```
   Indicating now the table can be read normally
   
   `.pyiceberg.yaml`:
   ```yaml
   catalog:
     default:
       type: glue
   ```


-- 
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 #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -476,6 +483,217 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    return visit_arrow_schema(schema, _ConvertToIceberg())
+
+
+def visit_arrow_schema(obj: pa.Schema, visitor: ArrowSchemaVisitor[T]) -> Schema:
+    struct_results = []
+    for i in range(len(obj.names)):
+        field = obj.field(i)
+        visitor.before_field(field)
+        struct_result = visit_arrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.schema(obj, struct_results)
+
+
+def visit_arrow(obj: pa.DataType, visitor: ArrowSchemaVisitor[T]) -> T:
+    if pa.types.is_struct(obj):
+        return visit_arrow_struct(obj, visitor)
+    elif pa.types.is_list(obj):
+        return visit_arrow_list(obj, visitor)
+    elif pa.types.is_map(obj):
+        return visit_arrow_map(obj, visitor)
+    else:
+        return visit_arrow_primitive(obj, visitor)
+
+
+def visit_arrow_struct(obj: pa.DataType, visitor: ArrowSchemaVisitor[T]) -> T:
+    if not pa.types.is_struct(obj):
+        raise TypeError(f"Expected struct type, got {type(obj)}")
+    obj = cast(pa.StructType, obj)
+    struct_results = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_arrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+def visit_arrow_list(obj: pa.DataType, visitor: ArrowSchemaVisitor[T]) -> T:
+    if not pa.types.is_list(obj):
+        raise TypeError(f"Expected list type, got {type(obj)}")
+    obj = cast(pa.ListType, obj)
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_arrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+def visit_arrow_map(obj: pa.DataType, visitor: ArrowSchemaVisitor[T]) -> T:
+    if not pa.types.is_map(obj):
+        raise TypeError(f"Expected map type, got {type(obj)}")
+    obj = cast(pa.MapType, obj)
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_arrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_arrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+def visit_arrow_primitive(obj: pa.DataType, visitor: ArrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class ArrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""
+
+    def after_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a list element."""
+
+    def before_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map key."""
+
+    def after_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map key."""
+
+    def before_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map value."""
+
+    def after_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map value."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[T]) -> Schema:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[T]) -> T:
+        """visit a struct"""
+
+    @abstractmethod
+    def list(self, list_type: pa.ListType, element_result: T) -> T:
+        """visit a list"""
+
+    @abstractmethod
+    def map(self, map_type: pa.MapType, key_result: T, value_result: T) -> T:
+        """visit a map"""
+
+    @abstractmethod
+    def primitive(self, primitive: pa.DataType) -> T:
+        """visit a primitive type"""
+
+
+def _get_field_id(field: pa.Field) -> int:
+    field_metadata = {k.decode(): v.decode() for k, v in field.metadata.items()}
+    if field_id := field_metadata.get("PARQUET:field_id"):
+        return field_id
+    raise ValueError(f"Field {field.name} does not have a field_id")
+
+
+class _ConvertToIceberg(ArrowSchemaVisitor[IcebergType], ABC):
+    def schema(self, schema: pa.Schema, field_results: List[IcebergType]) -> Schema:
+        fields = []
+        for i in range(len(schema.names)):
+            field = schema.field(i)
+            field_id = _get_field_id(field)
+            field_type = field_results[i]
+            if field_id is not None and field_type is not None:

Review Comment:
   Since `field_id` returns an `int`, I think we can just assume that it is non-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 #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -358,14 +368,17 @@ def field(self, field: NestedField, field_result: pa.DataType) -> pa.Field:
             name=field.name,
             type=field_result,
             nullable=field.optional,

Review Comment:
   Missed this one. I looked into this earlier, but it isn't really trivial to fix this. It is part of the Cython generator that generates docs, which are then interpreted by pycharm.



-- 
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 pull request #6997: Python: Infer Iceberg schema from the Parquet file

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

   Thanks, @JonasJ-ap! Great to have this improvement in.


-- 
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 #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -486,6 +499,207 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results: List[Optional[IcebergType]] = []
+    for field in schema:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(pa.DataType): An instance of a Schema or an IcebergType
+        visitor (PyArrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    struct_results: List[Optional[T]] = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got: {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyArrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""
+
+    def after_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a list element."""
+
+    def before_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map key."""
+
+    def after_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map key."""
+
+    def before_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map value."""
+
+    def after_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map value."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[Optional[T]]) -> Schema:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[Optional[T]]) -> T:

Review Comment:
   Why doesn't this return `Optional[T]` when the other methods do?



-- 
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] JonasJ-ap commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6997:
URL: https://github.com/apache/iceberg/pull/6997#discussion_r1125299171


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -476,6 +483,217 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    return visit_arrow_schema(schema, _ConvertToIceberg())
+
+
+def visit_arrow_schema(obj: pa.Schema, visitor: ArrowSchemaVisitor[T]) -> Schema:
+    struct_results = []
+    for i in range(len(obj.names)):
+        field = obj.field(i)
+        visitor.before_field(field)
+        struct_result = visit_arrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.schema(obj, struct_results)
+
+
+def visit_arrow(obj: pa.DataType, visitor: ArrowSchemaVisitor[T]) -> T:
+    if pa.types.is_struct(obj):
+        return visit_arrow_struct(obj, visitor)
+    elif pa.types.is_list(obj):
+        return visit_arrow_list(obj, visitor)
+    elif pa.types.is_map(obj):
+        return visit_arrow_map(obj, visitor)
+    else:
+        return visit_arrow_primitive(obj, visitor)
+
+
+def visit_arrow_struct(obj: pa.DataType, visitor: ArrowSchemaVisitor[T]) -> T:
+    if not pa.types.is_struct(obj):
+        raise TypeError(f"Expected struct type, got {type(obj)}")
+    obj = cast(pa.StructType, obj)
+    struct_results = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_arrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+def visit_arrow_list(obj: pa.DataType, visitor: ArrowSchemaVisitor[T]) -> T:
+    if not pa.types.is_list(obj):
+        raise TypeError(f"Expected list type, got {type(obj)}")
+    obj = cast(pa.ListType, obj)
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_arrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+def visit_arrow_map(obj: pa.DataType, visitor: ArrowSchemaVisitor[T]) -> T:
+    if not pa.types.is_map(obj):
+        raise TypeError(f"Expected map type, got {type(obj)}")
+    obj = cast(pa.MapType, obj)
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_arrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_arrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+def visit_arrow_primitive(obj: pa.DataType, visitor: ArrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class ArrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""
+
+    def after_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a list element."""
+
+    def before_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map key."""
+
+    def after_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map key."""
+
+    def before_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map value."""
+
+    def after_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map value."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[T]) -> Schema:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[T]) -> T:
+        """visit a struct"""
+
+    @abstractmethod
+    def list(self, list_type: pa.ListType, element_result: T) -> T:
+        """visit a list"""
+
+    @abstractmethod
+    def map(self, map_type: pa.MapType, key_result: T, value_result: T) -> T:
+        """visit a map"""
+
+    @abstractmethod
+    def primitive(self, primitive: pa.DataType) -> T:
+        """visit a primitive type"""
+
+
+def _get_field_id(field: pa.Field) -> int:
+    field_metadata = {k.decode(): v.decode() for k, v in field.metadata.items()}
+    if field_id := field_metadata.get("PARQUET:field_id"):
+        return field_id

Review Comment:
   Thank you for pointing this out. It was a bug that I introduced in my last minute refactor. It seems some implicit happen when it is passed into the iceberg NestedField. I changed it to `int(field_id)` to make it explicitly.
   
   I added two unit tests based on the existing schema fixture. I will post a comment below for more discussion



-- 
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 #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -356,14 +366,19 @@ def field(self, field: NestedField, field_result: pa.DataType) -> pa.Field:
             name=field.name,
             type=field_result,
             nullable=field.optional,
-            metadata={"doc": field.doc, "id": str(field.field_id)} if field.doc else {},
+            metadata={PYTHON_DOC.decode(): field.doc, PYTHON_FIELD_ID.decode(): str(field.field_id)}
+            if field.doc
+            else {PYTHON_FIELD_ID.decode(): str(field.field_id)},

Review Comment:
   I agree that `.endswith(FIELD_ID)` is too flexible. I'd rather just have a list of keys that we check for (and also in order of preference) as in the latter example.



-- 
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 #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -358,14 +368,17 @@ def field(self, field: NestedField, field_result: pa.DataType) -> pa.Field:
             name=field.name,
             type=field_result,
             nullable=field.optional,
-            metadata={"doc": field.doc, "id": str(field.field_id)} if field.doc else {},
+            metadata={FIELD_DOC: field.doc, FIELD_ID: str(field.field_id)} if field.doc else {FIELD_ID: str(field.field_id)},
         )
 
-    def list(self, _: ListType, element_result: pa.DataType) -> pa.DataType:
-        return pa.list_(value_type=element_result)
+    def list(self, list_type: ListType, element_result: pa.DataType) -> pa.DataType:
+        element_field = self.field(list_type.element_field, element_result)
+        return pa.list_(value_type=element_field)

Review Comment:
   I just checked, and they accept a field. You can see it in the code itself: https://github.com/apache/arrow/blob/28c68128ba7fb9702ce9f02529f3d1de61ffeac1/python/pyarrow/types.pxi#L4339-L4350
   
   Create a PR to fix this: https://github.com/apache/arrow/pull/35336



-- 
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 #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -476,6 +483,217 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:

Review Comment:
   I think this is a great start, but would suggest doing it the same way as the Iceberg Schema visitor:
   
   https://github.com/apache/iceberg/blob/d42d1e89c0616c203f7ad29f002811ddd440e14f/python/pyiceberg/schema.py#L710-L774
   
   Here we use the `@singledispatch` library to automatically call the right method based on the first argument.
   
   
   ```python
   def visit_arrow_list(obj: pa.DataType, visitor: ArrowSchemaVisitor[T]) -> T:
       if not pa.types.is_list(obj):
           raise TypeError(f"Expected list type, got {type(obj)}")
       obj = cast(pa.ListType, obj)
       visitor.before_list_element(obj.value_field)
       list_result = visit_arrow(obj.value_field.type, visitor)
       visitor.after_list_element(obj.value_field)
       return visitor.list(obj, list_result)
   ```
   
   Would become:
   ```python
   @visit_pyarrow.register(pa.lib.ListType)
   def visit_arrow_list(obj: pa.lib.ListType, visitor: ArrowSchemaVisitor[T]) -> T:
       visitor.before_list_element(obj.value_field)
       list_result = visit_arrow(obj.value_field.type, visitor)
       visitor.after_list_element(obj.value_field)
       return visitor.list(obj, list_result)
   ```
   
   This way we don't have to do all the checking, and we push this down to `dispatch`. I did some benchmarking, and it is also faster than trying to implement this ourselves (pushed down to the C level).
   
   The catch all for the primitive would then become the `DataType` one:
   ```
   >>> type(pa.int32())
   <class 'pyarrow.lib.DataType'>
   ```



-- 
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 #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -476,6 +483,202 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results = []
+    for i in range(len(schema.names)):
+        field = schema.field(i)
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(Schema | IcebergType): An instance of a Schema or an IcebergType
+        visitor (PyarrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    struct_results = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyarrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""
+
+    def after_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a list element."""
+
+    def before_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map key."""
+
+    def after_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map key."""
+
+    def before_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map value."""
+
+    def after_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map value."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[T]) -> Schema:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[T]) -> T:
+        """visit a struct"""
+
+    @abstractmethod
+    def list(self, list_type: pa.ListType, element_result: T) -> T:
+        """visit a list"""
+
+    @abstractmethod
+    def map(self, map_type: pa.MapType, key_result: T, value_result: T) -> T:
+        """visit a map"""
+
+    @abstractmethod
+    def primitive(self, primitive: pa.DataType) -> T:
+        """visit a primitive type"""
+
+
+def _get_field_id(field: pa.Field) -> int:
+    if field.metadata is not None:
+        field_metadata = {k.decode(): v.decode() for k, v in field.metadata.items()}
+        if field_id := field_metadata.get("PARQUET:field_id"):
+            return int(field_id)
+    raise ValueError(f"Field {field.name} does not have a field_id")
+
+
+class _ConvertToIceberg(PyarrowSchemaVisitor[IcebergType], ABC):
+    def schema(self, schema: pa.Schema, field_results: List[IcebergType]) -> Schema:
+        fields = []
+        for i in range(len(schema.names)):
+            field = schema.field(i)
+            field_id = _get_field_id(field)
+            field_type = field_results[i]
+            if field_type is not None:
+                fields.append(NestedField(field_id, field.name, field_type, required=not field.nullable))
+        return Schema(*fields)
+
+    def struct(self, struct: pa.StructType, field_results: List[IcebergType]) -> IcebergType:
+        fields = []
+        for i in range(struct.num_fields):
+            field = struct[i]
+            field_id = _get_field_id(field)
+            # may need to check doc strings

Review Comment:
   I don't think that this should assume how the converted schema will be used. It could easily be used to create a table, in which case you'd want to have docstrings.



-- 
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 #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -358,14 +368,17 @@ def field(self, field: NestedField, field_result: pa.DataType) -> pa.Field:
             name=field.name,
             type=field_result,
             nullable=field.optional,

Review Comment:
   @Fokko, looks like there's an issue with pyarrow here. Python gives a warning that this is not an argument to the function (which has `bool_nullable`) but that's probably not correct because the Python is a stub. Might be something we can fix upstream in pyarrow.



-- 
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] JonasJ-ap commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6997:
URL: https://github.com/apache/iceberg/pull/6997#discussion_r1174077806


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -358,14 +368,17 @@ def field(self, field: NestedField, field_result: pa.DataType) -> pa.Field:
             name=field.name,
             type=field_result,
             nullable=field.optional,
-            metadata={"doc": field.doc, "id": str(field.field_id)} if field.doc else {},
+            metadata={FIELD_DOC: field.doc, FIELD_ID: str(field.field_id)} if field.doc else {FIELD_ID: str(field.field_id)},
         )
 
-    def list(self, _: ListType, element_result: pa.DataType) -> pa.DataType:
-        return pa.list_(value_type=element_result)
+    def list(self, list_type: ListType, element_result: pa.DataType) -> pa.DataType:
+        element_field = self.field(list_type.element_field, element_result)
+        return pa.list_(value_type=element_field)

Review Comment:
   I think this is also a bug in pyarrow, similar to the above one. 
   Based on examples in this upstream issue: [[Python] MapType comparison ignores nullable flag of item_field · Issue #33574 · apache/arrow (github.com)](https://github.com/apache/arrow/issues/33574) `pa.map_` accepts `pa.field` as its key_value and map_value. I also confirmed this by looking at the underlying C++ constructor https://github.com/apache/arrow/blob/388f3a88c64750b3db77c5ca044ceae750e8db85/cpp/src/arrow/type.h#L1006-L1014
   
   In terms of this PR, I think this test indicates the `pa.map_` can accept `pa.field` as input  https://github.com/apache/iceberg/pull/6997/files#diff-320245c4e67f5814708bcf8aea113471b5d2e51e0e56b3eaae1a3bc1dfcbbfe7R303-R357
   
   Please correct me if I misunderstand something. Overall I think it is reasonable for `map_` to accept `DataType | Field` just as `list_` does



-- 
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] JonasJ-ap commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6997:
URL: https://github.com/apache/iceberg/pull/6997#discussion_r1131185116


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -476,6 +483,202 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results = []
+    for i in range(len(schema.names)):
+        field = schema.field(i)
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(Schema | IcebergType): An instance of a Schema or an IcebergType
+        visitor (PyarrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    struct_results = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyarrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""
+
+    def after_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a list element."""
+
+    def before_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map key."""
+
+    def after_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map key."""
+
+    def before_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map value."""
+
+    def after_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map value."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[T]) -> Schema:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[T]) -> T:
+        """visit a struct"""
+
+    @abstractmethod
+    def list(self, list_type: pa.ListType, element_result: T) -> T:
+        """visit a list"""
+
+    @abstractmethod
+    def map(self, map_type: pa.MapType, key_result: T, value_result: T) -> T:
+        """visit a map"""
+
+    @abstractmethod
+    def primitive(self, primitive: pa.DataType) -> T:
+        """visit a primitive type"""
+
+
+def _get_field_id(field: pa.Field) -> int:
+    if field.metadata is not None:
+        field_metadata = {k.decode(): v.decode() for k, v in field.metadata.items()}
+        if field_id := field_metadata.get("PARQUET:field_id"):
+            return int(field_id)
+    raise ValueError(f"Field {field.name} does not have a field_id")
+
+
+class _ConvertToIceberg(PyarrowSchemaVisitor[IcebergType], ABC):
+    def schema(self, schema: pa.Schema, field_results: List[IcebergType]) -> Schema:
+        fields = []
+        for i in range(len(schema.names)):
+            field = schema.field(i)
+            field_id = _get_field_id(field)
+            field_type = field_results[i]
+            if field_type is not None:
+                fields.append(NestedField(field_id, field.name, field_type, required=not field.nullable))
+        return Schema(*fields)
+
+    def struct(self, struct: pa.StructType, field_results: List[IcebergType]) -> IcebergType:
+        fields = []
+        for i in range(struct.num_fields):
+            field = struct[i]
+            field_id = _get_field_id(field)
+            # may need to check doc strings
+            field_type = field_results[i]
+            if field_type is not None:
+                fields.append(NestedField(field_id, field.name, field_type, required=not field.nullable))
+        return StructType(*fields)
+
+    def list(self, list_type: pa.ListType, element_result: IcebergType) -> IcebergType:
+        element_field = list_type.value_field
+        element_id = _get_field_id(element_field)
+        if element_result is not None:

Review Comment:
   Thanks for you clarification. I refactored the visitor to accommodate the change.



-- 
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] JonasJ-ap commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6997:
URL: https://github.com/apache/iceberg/pull/6997#discussion_r1133182512


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -356,14 +366,19 @@ def field(self, field: NestedField, field_result: pa.DataType) -> pa.Field:
             name=field.name,
             type=field_result,
             nullable=field.optional,
-            metadata={"doc": field.doc, "id": str(field.field_id)} if field.doc else {},
+            metadata={PYTHON_DOC.decode(): field.doc, PYTHON_FIELD_ID.decode(): str(field.field_id)}
+            if field.doc
+            else {PYTHON_FIELD_ID.decode(): str(field.field_id)},

Review Comment:
   Thank you very much for such a detailed explanation. I was thinking of the "pre-processing PyArrow schema" approach mentioned by Ryan. But now I see the problem of the prefix and I agree that we should leave name mapping for another PR. 
   
   I refactored the `_get_field_id_and_doc` to search for key ended with `id` such that both PARQUET and ORC could be handled. I also checked the AVRO format and found that the related key name is `field-id` so it also fits:
   ```python
   (base) ➜  user_id_bucket=2 avro-tools getschema 20230312_025043_00060_6msjy-bf8f6d3b-7e93-4f25-8a77-d0c183d6771a.avro
   23/03/11 21:55:31 WARN util.NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
   {
     "type" : "record",
     "name" : "table",
     "fields" : [ {
       "name" : "user_id",
       "type" : [ "null", "int" ],
       "default" : null,
       "field-id" : 1
     }
   ```
   Please let me know if something like `key.decode().endswith(FIELD_ID)` is too flexible in this case.
   



-- 
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 #6997: Python: Infer Iceberg schema from the Parquet file

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

   @JonasJ-ap I think we're good. Thanks so much for this PR and the refactoring of the tests. Let's @rdblue some time to hear if he has any final thoughts before merging.


-- 
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] JonasJ-ap commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6997:
URL: https://github.com/apache/iceberg/pull/6997#discussion_r1133368841


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -484,6 +495,203 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results: List[Optional[IcebergType]] = []
+    for field in schema:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(pa.DataType): An instance of a Schema or an IcebergType
+        visitor (PyArrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    struct_results: List[Optional[T]] = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyArrowSchemaVisitor(Generic[T], ABC):

Review Comment:
   Thank you for your suggestion. I think you mean the `_ConvertToIceberg` class downwards which is a conrete implementation of the abstract class `PyArrowSchemaVisitor`. So I removed `ABC` from `_ConvertToIceberg` class:
   ```Python
   class _ConvertToIceberg(PyArrowSchemaVisitor[IcebergType]):
   ```



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -484,6 +495,203 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results: List[Optional[IcebergType]] = []
+    for field in schema:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(pa.DataType): An instance of a Schema or an IcebergType
+        visitor (PyArrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    struct_results: List[Optional[T]] = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyArrowSchemaVisitor(Generic[T], ABC):

Review Comment:
   Thank you for your suggestion. I think you mean the `_ConvertToIceberg` class downwards which is a concrete implementation of the abstract class `PyArrowSchemaVisitor`. So I removed `ABC` from `_ConvertToIceberg` class:
   ```Python
   class _ConvertToIceberg(PyArrowSchemaVisitor[IcebergType]):
   ```



-- 
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 #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/tests/conftest.py:
##########
@@ -116,6 +117,17 @@ def table_schema_simple() -> Schema:
     )
 
 
+@pytest.fixture(scope="session")
+def pyarrow_schema_simple() -> pa.Schema:

Review Comment:
   I think these tests make too many assumptions about how the Parquet files will look. Have you verified that a real Parquet file produced by Java can be read this way?



-- 
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] JonasJ-ap commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6997:
URL: https://github.com/apache/iceberg/pull/6997#discussion_r1125317095


##########
python/tests/io/test_pyarrow.py:
##########
@@ -1130,3 +1131,15 @@ def test_projection_filter_on_unknown_field(schema_int_str: Schema, file_int_str
         _ = project(schema, [file_int_str], GreaterThan("unknown_field", "1"), schema_int_str)
 
     assert "Could not find field with name unknown_field, case_sensitive=True" in str(exc_info.value)
+
+
+def test_pyarrow_to_schema_simple(table_schema_simple: Schema, pyarrow_schema_simple: pa.Schema) -> None:
+    actual = str(pyarrow_to_schema(pyarrow_schema_simple))
+    expected = str(table_schema_simple)
+    assert actual == expected
+
+
+def test_pyarrow_to_schema_nested(table_schema_nested: Schema, pyarrow_schema_nested: pa.Schema) -> None:
+    actual = str(pyarrow_to_schema(pyarrow_schema_nested))

Review Comment:
   Initially, I was thinking of using `schema_to_pyarrow` and `pyarrow_to_schema` to do a round conversion to verify the correctness. But I found that `schema_to_pyarrow` does not assign field id to the metadata of list element and map key/value.
   https://github.com/apache/iceberg/blob/ad0f04dd90cf42011535207e661924c3b0456f1c/python/pyiceberg/io/pyarrow.py#L354-L359
   may need to be changed to 
   ```python
       def list(self, list_type: ListType, element_result: pa.DataType) -> pa.DataType:
           element_field = self.field(list_type.element_field, element_result)
           return pa.list_(value_type=element_field)
   
       def map(self, map_type: MapType, key_result: pa.DataType, value_result: pa.DataType) -> pa.DataType:
           key_field = self.field(map_type.key_field, key_result)
           value_field = self.field(map_type.value_field, value_result)
           return pa.map_(key_type=key_field, item_type=value_field)
   ```
   to preserve the field id in the metadata
   
   Based on my understanding, such change will only benefit the unit test at current stage. So I choose to not apply the change and hardcode a pyarrow schema fixture in `conftest.py` for testing instead.



##########
python/tests/io/test_pyarrow.py:
##########
@@ -1130,3 +1131,15 @@ def test_projection_filter_on_unknown_field(schema_int_str: Schema, file_int_str
         _ = project(schema, [file_int_str], GreaterThan("unknown_field", "1"), schema_int_str)
 
     assert "Could not find field with name unknown_field, case_sensitive=True" in str(exc_info.value)
+
+
+def test_pyarrow_to_schema_simple(table_schema_simple: Schema, pyarrow_schema_simple: pa.Schema) -> None:
+    actual = str(pyarrow_to_schema(pyarrow_schema_simple))
+    expected = str(table_schema_simple)
+    assert actual == expected
+
+
+def test_pyarrow_to_schema_nested(table_schema_nested: Schema, pyarrow_schema_nested: pa.Schema) -> None:
+    actual = str(pyarrow_to_schema(pyarrow_schema_nested))
+    expected = str(table_schema_nested)

Review Comment:
   I used `str` instead of `repr` because `repr` exposes `schema_id=1,identifier_field_ids=[1]` which is not handled by the `pyarrow_to_schema` visitor and I think the visitor need not handle these since the schema generated by the visitor will not be put into any table metadata file.
   
   Please correct me if I misunderstand something.



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -476,6 +483,202 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results = []
+    for i in range(len(schema.names)):
+        field = schema.field(i)
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(Schema | IcebergType): An instance of a Schema or an IcebergType
+        visitor (PyarrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    struct_results = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyarrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""
+
+    def after_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a list element."""
+
+    def before_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map key."""
+
+    def after_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map key."""
+
+    def before_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map value."""
+
+    def after_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map value."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[T]) -> Schema:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[T]) -> T:
+        """visit a struct"""
+
+    @abstractmethod
+    def list(self, list_type: pa.ListType, element_result: T) -> T:
+        """visit a list"""
+
+    @abstractmethod
+    def map(self, map_type: pa.MapType, key_result: T, value_result: T) -> T:
+        """visit a map"""
+
+    @abstractmethod
+    def primitive(self, primitive: pa.DataType) -> T:
+        """visit a primitive type"""
+
+
+def _get_field_id(field: pa.Field) -> int:
+    if field.metadata is not None:
+        field_metadata = {k.decode(): v.decode() for k, v in field.metadata.items()}
+        if field_id := field_metadata.get("PARQUET:field_id"):
+            return int(field_id)
+    raise ValueError(f"Field {field.name} does not have a field_id")
+
+
+class _ConvertToIceberg(PyarrowSchemaVisitor[IcebergType], ABC):
+    def schema(self, schema: pa.Schema, field_results: List[IcebergType]) -> Schema:
+        fields = []
+        for i in range(len(schema.names)):
+            field = schema.field(i)
+            field_id = _get_field_id(field)
+            field_type = field_results[i]
+            if field_type is not None:
+                fields.append(NestedField(field_id, field.name, field_type, required=not field.nullable))
+        return Schema(*fields)
+
+    def struct(self, struct: pa.StructType, field_results: List[IcebergType]) -> IcebergType:
+        fields = []
+        for i in range(struct.num_fields):
+            field = struct[i]
+            field_id = _get_field_id(field)
+            # may need to check doc strings

Review Comment:
   I re-consider this and think we do not need to fetch the optinal doc from the metadata as the inferred schema is only used to determine the correspondance between field name and field id. 



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -476,6 +483,202 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results = []
+    for i in range(len(schema.names)):
+        field = schema.field(i)
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(Schema | IcebergType): An instance of a Schema or an IcebergType
+        visitor (PyarrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    struct_results = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyarrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""
+
+    def after_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a list element."""
+
+    def before_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map key."""
+
+    def after_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map key."""
+
+    def before_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map value."""
+
+    def after_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map value."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[T]) -> Schema:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[T]) -> T:
+        """visit a struct"""
+
+    @abstractmethod
+    def list(self, list_type: pa.ListType, element_result: T) -> T:
+        """visit a list"""
+
+    @abstractmethod
+    def map(self, map_type: pa.MapType, key_result: T, value_result: T) -> T:
+        """visit a map"""
+
+    @abstractmethod
+    def primitive(self, primitive: pa.DataType) -> T:
+        """visit a primitive type"""
+
+
+def _get_field_id(field: pa.Field) -> int:
+    if field.metadata is not None:
+        field_metadata = {k.decode(): v.decode() for k, v in field.metadata.items()}
+        if field_id := field_metadata.get("PARQUET:field_id"):

Review Comment:
   Just to Confirm: `PARQUET:field_id` is the only key name in the metadata representing the field id and we do not need to check other similar names like `id`,`field_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 pull request #6997: Python: Infer Iceberg schema from the Parquet file

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

   @sheinbergon, I just need to review this. It should be included in 0.4.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 #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -476,6 +483,202 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results = []
+    for i in range(len(schema.names)):
+        field = schema.field(i)
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(Schema | IcebergType): An instance of a Schema or an IcebergType
+        visitor (PyarrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    struct_results = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyarrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""
+
+    def after_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a list element."""
+
+    def before_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map key."""
+
+    def after_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map key."""
+
+    def before_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map value."""
+
+    def after_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map value."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[T]) -> Schema:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[T]) -> T:
+        """visit a struct"""
+
+    @abstractmethod
+    def list(self, list_type: pa.ListType, element_result: T) -> T:
+        """visit a list"""
+
+    @abstractmethod
+    def map(self, map_type: pa.MapType, key_result: T, value_result: T) -> T:
+        """visit a map"""
+
+    @abstractmethod
+    def primitive(self, primitive: pa.DataType) -> T:
+        """visit a primitive type"""
+
+
+def _get_field_id(field: pa.Field) -> int:
+    if field.metadata is not None:
+        field_metadata = {k.decode(): v.decode() for k, v in field.metadata.items()}
+        if field_id := field_metadata.get("PARQUET:field_id"):
+            return int(field_id)
+    raise ValueError(f"Field {field.name} does not have a field_id")
+
+
+class _ConvertToIceberg(PyarrowSchemaVisitor[IcebergType], ABC):
+    def schema(self, schema: pa.Schema, field_results: List[IcebergType]) -> Schema:
+        fields = []
+        for i in range(len(schema.names)):
+            field = schema.field(i)
+            field_id = _get_field_id(field)
+            field_type = field_results[i]
+            if field_type is not None:
+                fields.append(NestedField(field_id, field.name, field_type, required=not field.nullable))
+        return Schema(*fields)
+
+    def struct(self, struct: pa.StructType, field_results: List[IcebergType]) -> IcebergType:
+        fields = []
+        for i in range(struct.num_fields):
+            field = struct[i]
+            field_id = _get_field_id(field)
+            # may need to check doc strings
+            field_type = field_results[i]
+            if field_type is not None:
+                fields.append(NestedField(field_id, field.name, field_type, required=not field.nullable))
+        return StructType(*fields)
+
+    def list(self, list_type: pa.ListType, element_result: IcebergType) -> IcebergType:
+        element_field = list_type.value_field
+        element_id = _get_field_id(element_field)
+        if element_result is not None:
+            return ListType(element_id, element_result, element_required=not element_field.nullable)
+        raise ValueError(f"List type must have element field: {list_type}")
+
+    def map(self, map_type: pa.MapType, key_result: IcebergType, value_result: IcebergType) -> IcebergType:
+        key_field = map_type.key_field
+        key_id = _get_field_id(key_field)
+        value_field = map_type.item_field
+        value_id = _get_field_id(value_field)
+        if key_result is not None and value_result is not None:
+            return MapType(key_id, key_result, value_id, value_result, value_required=not value_field.nullable)
+        raise ValueError(f"Map type must have key and value fields: {map_type}")
+
+    def primitive(self, primitive: pa.DataType) -> IcebergType:
+        if pa.types.is_boolean(primitive):
+            return BooleanType()
+        elif pa.types.is_int32(primitive) or pa.types.is_uint32(primitive):
+            return IntegerType()
+        elif pa.types.is_int64(primitive) or pa.types.is_uint64(primitive):
+            return LongType()
+        elif pa.types.is_float32(primitive):
+            return FloatType()
+        elif pa.types.is_float64(primitive):
+            return DoubleType()
+        elif pa.types.is_decimal(primitive):
+            if isinstance(primitive, pa.Decimal256Type):
+                primitive = cast(pa.Decimal256Type, primitive)
+            else:
+                primitive = cast(pa.Decimal128Type, primitive)
+            return DecimalType(primitive.precision, primitive.scale)
+        elif pa.types.is_string(primitive):
+            return StringType()
+        elif pa.types.is_date(primitive):
+            return DateType()
+        elif pa.types.is_time(primitive):

Review Comment:
   Do we need to check time or timestamp precision?



-- 
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 #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/tests/io/test_pyarrow.py:
##########
@@ -1130,3 +1131,15 @@ def test_projection_filter_on_unknown_field(schema_int_str: Schema, file_int_str
         _ = project(schema, [file_int_str], GreaterThan("unknown_field", "1"), schema_int_str)
 
     assert "Could not find field with name unknown_field, case_sensitive=True" in str(exc_info.value)
+
+
+def test_pyarrow_to_schema_simple(table_schema_simple: Schema, pyarrow_schema_simple: pa.Schema) -> None:

Review Comment:
   Can we move tests to their own test file? We want to avoid the test files getting so large.



-- 
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] JonasJ-ap commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6997:
URL: https://github.com/apache/iceberg/pull/6997#discussion_r1133182512


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -356,14 +366,19 @@ def field(self, field: NestedField, field_result: pa.DataType) -> pa.Field:
             name=field.name,
             type=field_result,
             nullable=field.optional,
-            metadata={"doc": field.doc, "id": str(field.field_id)} if field.doc else {},
+            metadata={PYTHON_DOC.decode(): field.doc, PYTHON_FIELD_ID.decode(): str(field.field_id)}
+            if field.doc
+            else {PYTHON_FIELD_ID.decode(): str(field.field_id)},

Review Comment:
   Thank you very much for such a detailed explanation. I was thinking of the "pre-processing PyArrow schema" approach mentioned by Ryan. But now I see the problem of the prefix and I agree that we should leave name mapping for another PR. 
   
   I refactored the `_get_field_id_and_doc` to search for key ended with `id` such that both PARQUET and ORC could be handled. I also checked the AVRO format and found that the related key name is `field-id` so it also fits:
   ```python
   (base) ➜  user_id_bucket=2 avro-tools getschema 20230312_025043_00060_6msjy-bf8f6d3b-7e93-4f25-8a77-d0c183d6771a.avro
   23/03/11 21:55:31 WARN util.NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
   {
     "type" : "record",
     "name" : "table",
     "fields" : [ {
       "name" : "user_id",
       "type" : [ "null", "int" ],
       "default" : null,
       "field-id" : 1
     }
   ```
   Please let me know if something like `key.decode().endswith(FIELD_ID)` is too flexible in this case. (I'm assuming that the iceberg data files' field metadata does not contain too many items and there will only be one key associated with 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] Fokko commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -484,6 +495,203 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results: List[Optional[IcebergType]] = []
+    for field in schema:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(pa.DataType): An instance of a Schema or an IcebergType
+        visitor (PyArrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    struct_results: List[Optional[T]] = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got {type(obj)}")

Review Comment:
   Nit
   ```suggestion
           raise TypeError(f"Expected primitive type, got: {type(obj)}")
   ```



-- 
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 #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -486,6 +499,207 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results: List[Optional[IcebergType]] = []
+    for field in schema:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(pa.DataType): An instance of a Schema or an IcebergType
+        visitor (PyArrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    struct_results: List[Optional[T]] = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got: {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyArrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""
+
+    def after_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a list element."""
+
+    def before_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map key."""
+
+    def after_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map key."""
+
+    def before_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map value."""
+
+    def after_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map value."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[Optional[T]]) -> Schema:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[Optional[T]]) -> T:
+        """visit a struct"""
+
+    @abstractmethod
+    def list(self, list_type: pa.ListType, element_result: Optional[T]) -> Optional[T]:
+        """visit a list"""
+
+    @abstractmethod
+    def map(self, map_type: pa.MapType, key_result: Optional[T], value_result: Optional[T]) -> Optional[T]:
+        """visit a map"""
+
+    @abstractmethod
+    def primitive(self, primitive: pa.DataType) -> T:
+        """visit a primitive type"""
+
+
+def _get_field_id_and_doc(field: pa.Field) -> Tuple[Optional[int], Optional[str]]:
+    field_id = None
+    doc = None
+
+    for pyarrow_field_id_key in PYARROW_FIELD_ID_KEYS:
+        if field_id_str := field.metadata.get(pyarrow_field_id_key):
+            field_id = int(field_id_str.decode())
+            break
+
+    for pyarrow_doc_key in PYARROW_FIELD_DOC_KEYS:
+        if doc_str := field.metadata.get(pyarrow_doc_key):
+            doc = doc_str.decode()
+            break
+
+    return field_id, doc
+
+
+class _ConvertToIceberg(PyArrowSchemaVisitor[IcebergType]):
+    def schema(self, schema: pa.Schema, field_results: List[Optional[IcebergType]]) -> Schema:
+        fields = []
+        for i, field in enumerate(schema):
+            field_id, field_doc = _get_field_id_and_doc(field)
+            field_type = field_results[i]
+            if field_type is not None and field_id is not None:
+                fields.append(NestedField(field_id, field.name, field_type, required=not field.nullable, doc=field_doc))
+        return Schema(*fields)
+
+    def struct(self, struct: pa.StructType, field_results: List[Optional[IcebergType]]) -> IcebergType:
+        fields = []
+        for i, field in enumerate(struct):
+            field_id, field_doc = _get_field_id_and_doc(field)
+            field_type = field_results[i]
+            if field_type is not None and field_id is not None:
+                fields.append(NestedField(field_id, field.name, field_type, required=not field.nullable, doc=field_doc))
+        return StructType(*fields)
+
+    def list(self, list_type: pa.ListType, element_result: Optional[IcebergType]) -> Optional[IcebergType]:
+        element_field = list_type.value_field
+        element_id, _ = _get_field_id_and_doc(element_field)
+        if element_result is not None and element_id is not None:
+            return ListType(element_id, element_result, element_required=not element_field.nullable)
+        return None
+
+    def map(
+        self, map_type: pa.MapType, key_result: Optional[IcebergType], value_result: Optional[IcebergType]
+    ) -> Optional[IcebergType]:
+        key_field = map_type.key_field
+        key_id, _ = _get_field_id_and_doc(key_field)
+        value_field = map_type.item_field
+        value_id, _ = _get_field_id_and_doc(value_field)
+        if key_result is not None and value_result is not None and key_id is not None and value_id is not None:
+            return MapType(key_id, key_result, value_id, value_result, value_required=not value_field.nullable)
+        return None
+
+    def primitive(self, primitive: pa.DataType) -> IcebergType:
+        if pa.types.is_boolean(primitive):
+            return BooleanType()
+        elif pa.types.is_int32(primitive):
+            return IntegerType()
+        elif pa.types.is_int64(primitive):
+            return LongType()
+        elif pa.types.is_float32(primitive):
+            return FloatType()
+        elif pa.types.is_float64(primitive):
+            return DoubleType()
+        elif pa.types.is_decimal(primitive) and isinstance(primitive, pa.Decimal128Type):
+            primitive = cast(pa.Decimal128Type, primitive)
+            return DecimalType(primitive.precision, primitive.scale)
+        elif pa.types.is_string(primitive):
+            return StringType()
+        elif pa.types.is_date32(primitive):
+            return DateType()
+        elif pa.types.is_time(primitive):
+            if isinstance(primitive, pa.Time64Type) and primitive.unit == "us":
+                return TimeType()
+        elif pa.types.is_timestamp(primitive):
+            primitive = cast(pa.TimestampType, primitive)
+            if primitive.unit == "us" and primitive.tz == "UTC":

Review Comment:
   Should we also accept a zero offset, like `+00:00`?



-- 
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] JonasJ-ap commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6997:
URL: https://github.com/apache/iceberg/pull/6997#discussion_r1179626777


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -507,11 +709,8 @@ def _file_to_table(
         schema_raw = None
         if metadata := physical_schema.metadata:
             schema_raw = metadata.get(ICEBERG_SCHEMA)
-        if schema_raw is None:
-            raise ValueError(
-                "Iceberg schema is not embedded into the Parquet file, see https://github.com/apache/iceberg/issues/6505"
-            )
-        file_schema = Schema.parse_raw(schema_raw)
+        # TODO: if field_ids are not present, Name Mapping should be implemented to look them up in the table schema

Review Comment:
   I created the issue: https://github.com/apache/iceberg/issues/7451.
   
   But I am not sure if what is the proper way to raise exception in this case. Based on my understanding, name mapping is also needed if portion of parquet fields miss the field ids. However, in this case, `pyarrow_to_schema` can still generate a valid iceberg schema for the the rest of parquet fields. It seems we should not raise exception in this case.
   
   Should we only raise exception when no field id exist in the whole data file? I think we can also log some warning messages when a pyarrow field containing a field id. What do you think?
   



-- 
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 #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -476,6 +483,202 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results = []
+    for i in range(len(schema.names)):
+        field = schema.field(i)
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(Schema | IcebergType): An instance of a Schema or an IcebergType
+        visitor (PyarrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    struct_results = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyarrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""
+
+    def after_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a list element."""
+
+    def before_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map key."""
+
+    def after_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map key."""
+
+    def before_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map value."""
+
+    def after_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map value."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[T]) -> Schema:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[T]) -> T:
+        """visit a struct"""
+
+    @abstractmethod
+    def list(self, list_type: pa.ListType, element_result: T) -> T:
+        """visit a list"""
+
+    @abstractmethod
+    def map(self, map_type: pa.MapType, key_result: T, value_result: T) -> T:
+        """visit a map"""
+
+    @abstractmethod
+    def primitive(self, primitive: pa.DataType) -> T:
+        """visit a primitive type"""
+
+
+def _get_field_id(field: pa.Field) -> int:
+    if field.metadata is not None:
+        field_metadata = {k.decode(): v.decode() for k, v in field.metadata.items()}
+        if field_id := field_metadata.get("PARQUET:field_id"):
+            return int(field_id)
+    raise ValueError(f"Field {field.name} does not have a field_id")
+
+
+class _ConvertToIceberg(PyarrowSchemaVisitor[IcebergType], ABC):
+    def schema(self, schema: pa.Schema, field_results: List[IcebergType]) -> Schema:
+        fields = []
+        for i in range(len(schema.names)):
+            field = schema.field(i)
+            field_id = _get_field_id(field)
+            field_type = field_results[i]
+            if field_type is not None:
+                fields.append(NestedField(field_id, field.name, field_type, required=not field.nullable))
+        return Schema(*fields)
+
+    def struct(self, struct: pa.StructType, field_results: List[IcebergType]) -> IcebergType:
+        fields = []
+        for i in range(struct.num_fields):
+            field = struct[i]
+            field_id = _get_field_id(field)
+            # may need to check doc strings
+            field_type = field_results[i]
+            if field_type is not None:
+                fields.append(NestedField(field_id, field.name, field_type, required=not field.nullable))
+        return StructType(*fields)
+
+    def list(self, list_type: pa.ListType, element_result: IcebergType) -> IcebergType:
+        element_field = list_type.value_field
+        element_id = _get_field_id(element_field)
+        if element_result is not None:
+            return ListType(element_id, element_result, element_required=not element_field.nullable)
+        raise ValueError(f"List type must have element field: {list_type}")
+
+    def map(self, map_type: pa.MapType, key_result: IcebergType, value_result: IcebergType) -> IcebergType:
+        key_field = map_type.key_field
+        key_id = _get_field_id(key_field)
+        value_field = map_type.item_field
+        value_id = _get_field_id(value_field)
+        if key_result is not None and value_result is not None:
+            return MapType(key_id, key_result, value_id, value_result, value_required=not value_field.nullable)
+        raise ValueError(f"Map type must have key and value fields: {map_type}")
+
+    def primitive(self, primitive: pa.DataType) -> IcebergType:
+        if pa.types.is_boolean(primitive):
+            return BooleanType()
+        elif pa.types.is_int32(primitive) or pa.types.is_uint32(primitive):
+            return IntegerType()
+        elif pa.types.is_int64(primitive) or pa.types.is_uint64(primitive):
+            return LongType()
+        elif pa.types.is_float32(primitive):
+            return FloatType()
+        elif pa.types.is_float64(primitive):
+            return DoubleType()
+        elif pa.types.is_decimal(primitive):
+            if isinstance(primitive, pa.Decimal256Type):
+                primitive = cast(pa.Decimal256Type, primitive)
+            else:
+                primitive = cast(pa.Decimal128Type, primitive)
+            return DecimalType(primitive.precision, primitive.scale)
+        elif pa.types.is_string(primitive):
+            return StringType()
+        elif pa.types.is_date(primitive):
+            return DateType()
+        elif pa.types.is_time(primitive):

Review Comment:
   @Fokko, I think this could be a problem. We should make sure there are good tests for these types.
   
   The read code operates using PyIceberg schemas. So that is going to convert PyArrow to PyIceberg, prune columns, and the convert back to request a projection from PyArrow. If there is a lossy conversion then that will cause a problem. For example if a time type is in millis but is converted without that information, then the requested type will be a timestamp in micros and that may cause a problem.
   
   When looking at this, I found another problem. Rather than requesting a specific schema, the projected Iceberg schema is used to request top-level columns:
   
   ```python
           arrow_table = pq.read_table(
               source=fout,
               schema=parquet_schema,
               pre_buffer=True,
               buffer_size=8 * ONE_MEGABYTE,
               filters=pyarrow_filter,
               columns=[col.name for col in file_project_schema.columns],
           )
   ```
   
   I assume that's going to read full nested structures because the only information it has is the top-level column name. If that happens, I'm concerned that we're reading leaf columns that we don't actually need to. I think projection will correctly ignore them, but it would be nice to avoid reading them.



-- 
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 #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -476,6 +483,202 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results = []
+    for i in range(len(schema.names)):
+        field = schema.field(i)
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(Schema | IcebergType): An instance of a Schema or an IcebergType
+        visitor (PyarrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    struct_results = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyarrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""
+
+    def after_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a list element."""
+
+    def before_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map key."""
+
+    def after_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map key."""
+
+    def before_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map value."""
+
+    def after_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map value."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[T]) -> Schema:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[T]) -> T:
+        """visit a struct"""
+
+    @abstractmethod
+    def list(self, list_type: pa.ListType, element_result: T) -> T:
+        """visit a list"""
+
+    @abstractmethod
+    def map(self, map_type: pa.MapType, key_result: T, value_result: T) -> T:
+        """visit a map"""
+
+    @abstractmethod
+    def primitive(self, primitive: pa.DataType) -> T:
+        """visit a primitive type"""
+
+
+def _get_field_id(field: pa.Field) -> int:

Review Comment:
   I think this should return None when there is no ID so that the field can be skipped.



-- 
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] JonasJ-ap commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6997:
URL: https://github.com/apache/iceberg/pull/6997#discussion_r1131143159


##########
python/tests/io/test_pyarrow.py:
##########
@@ -1130,3 +1131,15 @@ def test_projection_filter_on_unknown_field(schema_int_str: Schema, file_int_str
         _ = project(schema, [file_int_str], GreaterThan("unknown_field", "1"), schema_int_str)
 
     assert "Could not find field with name unknown_field, case_sensitive=True" in str(exc_info.value)
+
+
+def test_pyarrow_to_schema_simple(table_schema_simple: Schema, pyarrow_schema_simple: pa.Schema) -> None:
+    actual = str(pyarrow_to_schema(pyarrow_schema_simple))
+    expected = str(table_schema_simple)
+    assert actual == expected
+
+
+def test_pyarrow_to_schema_nested(table_schema_nested: Schema, pyarrow_schema_nested: pa.Schema) -> None:
+    actual = str(pyarrow_to_schema(pyarrow_schema_nested))

Review Comment:
   Thank you for the testing. I applied this change to pyarrow.py



-- 
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] JonasJ-ap commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6997:
URL: https://github.com/apache/iceberg/pull/6997#discussion_r1131156819


##########
python/tests/io/test_pyarrow.py:
##########
@@ -1130,3 +1131,15 @@ def test_projection_filter_on_unknown_field(schema_int_str: Schema, file_int_str
         _ = project(schema, [file_int_str], GreaterThan("unknown_field", "1"), schema_int_str)
 
     assert "Could not find field with name unknown_field, case_sensitive=True" in str(exc_info.value)
+
+
+def test_pyarrow_to_schema_simple(table_schema_simple: Schema, pyarrow_schema_simple: pa.Schema) -> None:

Review Comment:
   Sure! I separated these tests into 4 files: `test_pyarrow`, `test_pyarrow_expression`, `test_pyarrow_projection`, and `test_pyarrow_projection`



-- 
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] JonasJ-ap commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6997:
URL: https://github.com/apache/iceberg/pull/6997#discussion_r1133139183


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -356,14 +366,19 @@ def field(self, field: NestedField, field_result: pa.DataType) -> pa.Field:
             name=field.name,
             type=field_result,
             nullable=field.optional,
-            metadata={"doc": field.doc, "id": str(field.field_id)} if field.doc else {},
+            metadata={PYTHON_DOC.decode(): field.doc, PYTHON_FIELD_ID.decode(): str(field.field_id)}
+            if field.doc
+            else {PYTHON_FIELD_ID.decode(): str(field.field_id)},

Review Comment:
   I made some chanes in the metadata field name for iceberg to pyarrow visitor: from `doc`, `id` to `PYTHON:field_doc` to `PYTHON:field_id`. My thought here is that with this change, we can make the name consistent with its source: `PARQUET` indicates that the field comes from the parquet file. `PYTHON` indicates that the field is inferred from the pyiceberg table schema. 
   
   The `_get_field_id_and_doc` will first search field labelled with `PARQUET` first and then search for fields labelled with `PYTHON`. The order is consistent with implementation of name mapping in the future as suggested by @rdblue in https://github.com/apache/iceberg/pull/6997#discussion_r1125740765
   
   @Fokko @rdblue May I ask what do you think about this change?



-- 
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 #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -507,11 +709,8 @@ def _file_to_table(
         schema_raw = None
         if metadata := physical_schema.metadata:
             schema_raw = metadata.get(ICEBERG_SCHEMA)
-        if schema_raw is None:
-            raise ValueError(
-                "Iceberg schema is not embedded into the Parquet file, see https://github.com/apache/iceberg/issues/6505"
-            )
-        file_schema = Schema.parse_raw(schema_raw)
+        # TODO: if field_ids are not present, Name Mapping should be implemented to look them up in the table schema

Review Comment:
   Should we create an issue for this, and still raise an exception?



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -486,6 +499,195 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    return visit_pyarrow(schema, visitor)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType | pa.Schema, visitor: PyArrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(pa.DataType): An instance of a Schema or an IcebergType
+        visitor (PyArrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.Schema)
+def _(obj: pa.Schema, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    struct_results: List[Optional[T]] = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.schema(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    struct_results: List[Optional[T]] = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_field(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_field(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_field(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_field(obj.key_field)
+    visitor.before_field(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_field(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got: {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyArrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[Optional[T]]) -> Optional[T]:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[Optional[T]]) -> Optional[T]:
+        """visit a struct"""
+
+    @abstractmethod
+    def list(self, list_type: pa.ListType, element_result: Optional[T]) -> Optional[T]:
+        """visit a list"""
+
+    @abstractmethod
+    def map(self, map_type: pa.MapType, key_result: Optional[T], value_result: Optional[T]) -> Optional[T]:
+        """visit a map"""
+
+    @abstractmethod
+    def primitive(self, primitive: pa.DataType) -> Optional[T]:
+        """visit a primitive type"""
+
+
+def _get_field_id(field: pa.Field) -> Optional[int]:
+    for pyarrow_field_id_key in PYARROW_FIELD_ID_KEYS:
+        if field_id_str := field.metadata.get(pyarrow_field_id_key):
+            return int(field_id_str.decode())
+    return None
+
+
+def _get_field_doc(field: pa.Field) -> Optional[str]:
+    for pyarrow_doc_key in PYARROW_FIELD_DOC_KEYS:
+        if doc_str := field.metadata.get(pyarrow_doc_key):
+            return doc_str.decode()
+    return None
+
+
+class _ConvertToIceberg(PyArrowSchemaVisitor[Union[IcebergType, Schema]]):
+    def schema(self, schema: pa.Schema, field_results: List[Optional[IcebergType]]) -> Schema:

Review Comment:
   I noticed that there is quite a bit of duplication in the schema and struct, how do you feel about:
   ```python
       def _convert_fields(self, arrow_fields: Iterable[pa.Field], field_results: List[Optional[IcebergType]]) -> List[NestedField]:
           fields = []
           for i, field in enumerate(arrow_fields):
               field_id = _get_field_id(field)
               field_doc = _get_field_doc(field)
               field_type = field_results[i]
               if field_type is not None and field_id is not None:
                   fields.append(NestedField(field_id, field.name, field_type, required=not field.nullable, doc=field_doc))
           return fields
   
       def schema(self, schema: pa.Schema, field_results: List[Optional[IcebergType]]) -> Schema:
           return Schema(*self._convert_fields(schema, field_results))
   
       def struct(self, struct: pa.StructType, field_results: List[Optional[IcebergType]]) -> IcebergType:
           return StructType(*self._convert_fields(struct, field_results))
   ```
   Ran the type checker and tests, and this works 👍🏻 



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -476,6 +483,202 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results = []
+    for i in range(len(schema.names)):
+        field = schema.field(i)
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(Schema | IcebergType): An instance of a Schema or an IcebergType
+        visitor (PyarrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    struct_results = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyarrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""
+
+    def after_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a list element."""
+
+    def before_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map key."""
+
+    def after_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map key."""
+
+    def before_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map value."""
+
+    def after_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map value."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[T]) -> Schema:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[T]) -> T:
+        """visit a struct"""
+
+    @abstractmethod
+    def list(self, list_type: pa.ListType, element_result: T) -> T:
+        """visit a list"""
+
+    @abstractmethod
+    def map(self, map_type: pa.MapType, key_result: T, value_result: T) -> T:
+        """visit a map"""
+
+    @abstractmethod
+    def primitive(self, primitive: pa.DataType) -> T:
+        """visit a primitive type"""
+
+
+def _get_field_id(field: pa.Field) -> int:
+    if field.metadata is not None:
+        field_metadata = {k.decode(): v.decode() for k, v in field.metadata.items()}
+        if field_id := field_metadata.get("PARQUET:field_id"):
+            return int(field_id)
+    raise ValueError(f"Field {field.name} does not have a field_id")
+
+
+class _ConvertToIceberg(PyarrowSchemaVisitor[IcebergType], ABC):
+    def schema(self, schema: pa.Schema, field_results: List[IcebergType]) -> Schema:
+        fields = []
+        for i in range(len(schema.names)):
+            field = schema.field(i)
+            field_id = _get_field_id(field)
+            field_type = field_results[i]
+            if field_type is not None:
+                fields.append(NestedField(field_id, field.name, field_type, required=not field.nullable))
+        return Schema(*fields)
+
+    def struct(self, struct: pa.StructType, field_results: List[IcebergType]) -> IcebergType:
+        fields = []
+        for i in range(struct.num_fields):
+            field = struct[i]
+            field_id = _get_field_id(field)
+            # may need to check doc strings
+            field_type = field_results[i]
+            if field_type is not None:
+                fields.append(NestedField(field_id, field.name, field_type, required=not field.nullable))
+        return StructType(*fields)
+
+    def list(self, list_type: pa.ListType, element_result: IcebergType) -> IcebergType:
+        element_field = list_type.value_field
+        element_id = _get_field_id(element_field)
+        if element_result is not None:
+            return ListType(element_id, element_result, element_required=not element_field.nullable)
+        raise ValueError(f"List type must have element field: {list_type}")
+
+    def map(self, map_type: pa.MapType, key_result: IcebergType, value_result: IcebergType) -> IcebergType:
+        key_field = map_type.key_field
+        key_id = _get_field_id(key_field)
+        value_field = map_type.item_field
+        value_id = _get_field_id(value_field)
+        if key_result is not None and value_result is not None:
+            return MapType(key_id, key_result, value_id, value_result, value_required=not value_field.nullable)
+        raise ValueError(f"Map type must have key and value fields: {map_type}")
+
+    def primitive(self, primitive: pa.DataType) -> IcebergType:
+        if pa.types.is_boolean(primitive):
+            return BooleanType()
+        elif pa.types.is_int32(primitive) or pa.types.is_uint32(primitive):
+            return IntegerType()
+        elif pa.types.is_int64(primitive) or pa.types.is_uint64(primitive):
+            return LongType()
+        elif pa.types.is_float32(primitive):
+            return FloatType()
+        elif pa.types.is_float64(primitive):
+            return DoubleType()
+        elif pa.types.is_decimal(primitive):
+            if isinstance(primitive, pa.Decimal256Type):
+                primitive = cast(pa.Decimal256Type, primitive)
+            else:
+                primitive = cast(pa.Decimal128Type, primitive)
+            return DecimalType(primitive.precision, primitive.scale)
+        elif pa.types.is_string(primitive):
+            return StringType()
+        elif pa.types.is_date(primitive):
+            return DateType()
+        elif pa.types.is_time(primitive):

Review Comment:
   If there is an Iceberg schema in the file, I think we can assume that it is written according to the spec:
   ![image](https://user-images.githubusercontent.com/1134248/234511353-ce691d90-4341-4369-a7ca-41f3bb33c8b7.png)
   
   With the current check, it is correct:
   ```python
           elif pa.types.is_time(primitive):
               if isinstance(primitive, pa.Time64Type) and primitive.unit == "us":
                   return TimeType()
   ```
   
   We could even simplify it:
   
   ```python
           elif isinstance(primitive, pa.Time64Type) and primitive.unit == "us":
               return TimeType()
   ```
   
   The tests are in place:
   
   ```python
   def test_pyarrow_time32_to_iceberg() -> None:
       pyarrow_type = pa.time32("ms")
       with pytest.raises(TypeError, match=re.escape("Unsupported type: time32[ms]")):
           visit_pyarrow(pyarrow_type, _ConvertToIceberg())
       pyarrow_type = pa.time32("s")
       with pytest.raises(TypeError, match=re.escape("Unsupported type: time32[s]")):
           visit_pyarrow(pyarrow_type, _ConvertToIceberg())
   
   
   def test_pyarrow_time64_us_to_iceberg() -> None:
       pyarrow_type = pa.time64("us")
       converted_iceberg_type = visit_pyarrow(pyarrow_type, _ConvertToIceberg())
       assert converted_iceberg_type == TimeType()
       assert visit(converted_iceberg_type, _ConvertToArrowSchema()) == pyarrow_type
   
   
   def test_pyarrow_time64_ns_to_iceberg() -> None:
       pyarrow_type = pa.time64("ns")
       with pytest.raises(TypeError, match=re.escape("Unsupported type: time64[ns]")):
           visit_pyarrow(pyarrow_type, _ConvertToIceberg())
   ```
   
   I think we can resolve this issue



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

To unsubscribe, e-mail: 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] JonasJ-ap commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6997:
URL: https://github.com/apache/iceberg/pull/6997#discussion_r1174077806


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -358,14 +368,17 @@ def field(self, field: NestedField, field_result: pa.DataType) -> pa.Field:
             name=field.name,
             type=field_result,
             nullable=field.optional,
-            metadata={"doc": field.doc, "id": str(field.field_id)} if field.doc else {},
+            metadata={FIELD_DOC: field.doc, FIELD_ID: str(field.field_id)} if field.doc else {FIELD_ID: str(field.field_id)},
         )
 
-    def list(self, _: ListType, element_result: pa.DataType) -> pa.DataType:
-        return pa.list_(value_type=element_result)
+    def list(self, list_type: ListType, element_result: pa.DataType) -> pa.DataType:
+        element_field = self.field(list_type.element_field, element_result)
+        return pa.list_(value_type=element_field)

Review Comment:
   I think this is also a bug in pyarrow, similar to the above one. 
   Based on examples in this upstream issue: [[Python] MapType comparison ignores nullable flag of item_field · Issue #33574 · apache/arrow (github.com)](https://github.com/apache/arrow/issues/33574) `pa.map_` accepts `pa.field` as its key_value and map_value. I also confirmed this by looking at the underlying C++ constructor https://github.com/apache/arrow/blob/388f3a88c64750b3db77c5ca044ceae750e8db85/cpp/src/arrow/type.h#L1006-L1014
   
   In terms of this PR, I think this test indicates the `pa.map_` can accept `pa.field` as input  https://github.com/apache/iceberg/pull/6997/files#diff-320245c4e67f5814708bcf8aea113471b5d2e51e0e56b3eaae1a3bc1dfcbbfe7R303-R357



-- 
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] JonasJ-ap commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6997:
URL: https://github.com/apache/iceberg/pull/6997#discussion_r1174074325


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -486,6 +499,207 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results: List[Optional[IcebergType]] = []
+    for field in schema:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)

Review Comment:
   Thank you for your suggestions. I refactored it into the `visti_pyarrow`



-- 
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] JonasJ-ap commented on pull request #6997: Python: Infer Iceberg schema from the Parquet file

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

   @rdblue Thanks for your review and suggestions. I will try to update the PR by this Friday.


-- 
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 #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/tests/io/test_pyarrow.py:
##########
@@ -18,73 +18,19 @@
 
 import os
 import tempfile
-from typing import Any, List, Optional

Review Comment:
   Can you move this refactor to a different PR? Then we can easily validate the refactor independently. For this PR, all that we should do is write tests in a separate test 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 #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -122,6 +126,12 @@
 ONE_MEGABYTE = 1024 * 1024
 BUFFER_SIZE = "buffer-size"
 ICEBERG_SCHEMA = b"iceberg.schema"
+FIELD_ID = "field_id"
+FIELD_DOC = "doc"

Review Comment:
   Should this constant name match its contents? That's what we would normally do so that code is 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 #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -356,14 +366,19 @@ def field(self, field: NestedField, field_result: pa.DataType) -> pa.Field:
             name=field.name,
             type=field_result,
             nullable=field.optional,
-            metadata={"doc": field.doc, "id": str(field.field_id)} if field.doc else {},
+            metadata={PYTHON_DOC.decode(): field.doc, PYTHON_FIELD_ID.decode(): str(field.field_id)}
+            if field.doc
+            else {PYTHON_FIELD_ID.decode(): str(field.field_id)},

Review Comment:
   I would probably get rid of the `PARQUET:` prefix when reading the data. It looks like this is something that PyArrow introduces. But when we write the fields again, we definitely don't want to have that prefix there, otherwise next time it would read as `PARQUET:PARQUET:field_id` etc :)
   
   I checked with duckdb, and it shows just `field_id`:
   ```
   ➜  Desktop duckdb
   v0.7.1 b00b93f0b1
   Enter ".help" for usage hints.
   Connected to a transient in-memory database.
   Use ".open FILENAME" to reopen on a persistent database.
   D SELECT * FROM PARQUET_SCHEMA('/Users/fokkodriesprong/Desktop/00003-4-5e18fe8b-532b-42ec-bdb9-0c7ba7078155-00001.parquet');
   ┌──────────────────────┬──────────────────────┬────────────┬─────────────┬───┬───────┬───────────┬──────────┬──────────────────────┐
   │      file_name       │         name         │    type    │ type_length │ … │ scale │ precision │ field_id │     logical_type     │
   │       varchar        │       varchar        │  varchar   │   varchar   │   │ int64 │   int64   │  int64   │       varchar        │
   ├──────────────────────┼──────────────────────┼────────────┼─────────────┼───┼───────┼───────────┼──────────┼──────────────────────┤
   │ /Users/fokkodriesp…  │ table                │ BOOLEAN    │ 0           │ … │     0 │         0 │        0 │                      │
   │ /Users/fokkodriesp…  │ VendorID             │ INT64      │ 0           │ … │     0 │         0 │        1 │                      │
   │ /Users/fokkodriesp…  │ tpep_pickup_datetime │ INT64      │ 0           │ … │     0 │         0 │        2 │ TimestampType(isAd…  │
   │ /Users/fokkodriesp…  │ tpep_dropoff_datet…  │ INT64      │ 0           │ … │     0 │         0 │        3 │ TimestampType(isAd…  │
   │ /Users/fokkodriesp…  │ passenger_count      │ DOUBLE     │ 0           │ … │     0 │         0 │        4 │                      │
   │ /Users/fokkodriesp…  │ trip_distance        │ DOUBLE     │ 0           │ … │     0 │         0 │        5 │                      │
   │ /Users/fokkodriesp…  │ RatecodeID           │ DOUBLE     │ 0           │ … │     0 │         0 │        6 │                      │
   │ /Users/fokkodriesp…  │ store_and_fwd_flag   │ BYTE_ARRAY │ 0           │ … │     0 │         0 │        7 │ StringType()         │
   │ /Users/fokkodriesp…  │ PULocationID         │ INT64      │ 0           │ … │     0 │         0 │        8 │                      │
   │ /Users/fokkodriesp…  │ DOLocationID         │ INT64      │ 0           │ … │     0 │         0 │        9 │                      │
   │ /Users/fokkodriesp…  │ payment_type         │ INT64      │ 0           │ … │     0 │         0 │       10 │                      │
   │ /Users/fokkodriesp…  │ fare_amount          │ DOUBLE     │ 0           │ … │     0 │         0 │       11 │                      │
   │ /Users/fokkodriesp…  │ extra                │ DOUBLE     │ 0           │ … │     0 │         0 │       12 │                      │
   │ /Users/fokkodriesp…  │ mta_tax              │ DOUBLE     │ 0           │ … │     0 │         0 │       13 │                      │
   │ /Users/fokkodriesp…  │ tip_amount           │ DOUBLE     │ 0           │ … │     0 │         0 │       14 │                      │
   │ /Users/fokkodriesp…  │ tolls_amount         │ DOUBLE     │ 0           │ … │     0 │         0 │       15 │                      │
   │ /Users/fokkodriesp…  │ improvement_surcha…  │ DOUBLE     │ 0           │ … │     0 │         0 │       16 │                      │
   │ /Users/fokkodriesp…  │ total_amount         │ DOUBLE     │ 0           │ … │     0 │         0 │       17 │                      │
   │ /Users/fokkodriesp…  │ congestion_surcharge │ DOUBLE     │ 0           │ … │     0 │         0 │       18 │                      │
   │ /Users/fokkodriesp…  │ airport_fee          │ DOUBLE     │ 0           │ … │     0 │         0 │       19 │                      │
   ├──────────────────────┴──────────────────────┴────────────┴─────────────┴───┴───────┴───────────┴──────────┴──────────────────────┤
   │ 20 rows                                                                                                     11 columns (8 shown) │
   └──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘
   ```
   
   I also double checked with orc, and there it is [`field.id`](https://github.com/apache/iceberg/blob/bc650a1cd586b9c2f8063851e6d583533435f31e/orc/src/main/java/org/apache/iceberg/orc/ORCSchemaUtil.java#L68):
   
   ```
   ➜  Desktop orc-tools meta 00006-220-98bb5767-2804-4986-bc98-f48e94365b84-00001.orc     
   log4j:WARN No appenders could be found for logger (org.apache.hadoop.util.Shell).
   log4j:WARN Please initialize the log4j system properly.
   log4j:WARN See http://logging.apache.org/log4j/1.2/faq.html#noconfig for more info.
   Processing data file 00006-220-98bb5767-2804-4986-bc98-f48e94365b84-00001.orc [length: 2229]
   Structure for 00006-220-98bb5767-2804-4986-bc98-f48e94365b84-00001.orc
   File Version: 0.12 with ORC_14 by ORC Java 1.7.8
   Rows: 7
   Compression: ZLIB
   Compression size: 262144
   Calendar: Julian/Gregorian
   Type: struct<VendorID:bigint,tpep_pickup_datetime:timestamp with local time zone,tpep_dropoff_datetime:timestamp with local time zone,passenger_count:double,trip_distance:double,RatecodeID:double,store_and_fwd_flag:string,PULocationID:bigint,DOLocationID:bigint,payment_type:bigint,fare_amount:double,extra:double,mta_tax:double,tip_amount:double,tolls_amount:double,improvement_surcharge:double,total_amount:double,congestion_surcharge:double,airport_fee:double>
   Attributes on root.VendorID
     iceberg.id: 1
     iceberg.long-type: LONG
     iceberg.required: false
   Attributes on root.tpep_pickup_datetime
     iceberg.id: 2
     iceberg.required: false
   Attributes on root.tpep_dropoff_datetime
     iceberg.id: 3
     iceberg.required: false
   Attributes on root.passenger_count
     iceberg.id: 4
     iceberg.required: false
   Attributes on root.trip_distance
     iceberg.id: 5
     iceberg.required: false
   Attributes on root.RatecodeID
     iceberg.id: 6
     iceberg.required: false
   Attributes on root.store_and_fwd_flag
     iceberg.id: 7
     iceberg.required: false
   Attributes on root.PULocationID
     iceberg.id: 8
     iceberg.long-type: LONG
     iceberg.required: false
   Attributes on root.DOLocationID
     iceberg.id: 9
     iceberg.long-type: LONG
     iceberg.required: false
   Attributes on root.payment_type
     iceberg.id: 10
     iceberg.long-type: LONG
     iceberg.required: false
   Attributes on root.fare_amount
     iceberg.id: 11
     iceberg.required: false
   Attributes on root.extra
     iceberg.id: 12
     iceberg.required: false
   Attributes on root.mta_tax
     iceberg.id: 13
     iceberg.required: false
   Attributes on root.tip_amount
     iceberg.id: 14
     iceberg.required: false
   Attributes on root.tolls_amount
     iceberg.id: 15
     iceberg.required: false
   Attributes on root.improvement_surcharge
     iceberg.id: 16
     iceberg.required: false
   Attributes on root.total_amount
     iceberg.id: 17
     iceberg.required: false
   Attributes on root.congestion_surcharge
     iceberg.id: 18
     iceberg.required: false
   Attributes on root.airport_fee
     iceberg.id: 19
     iceberg.required: false
   ```
   
   So probably we need to be flexible with the keys anyway. Looking at Ryan's original comment: https://github.com/apache/iceberg/pull/6997#discussion_r1125740765
   
   I think what he means is that the first check if there are IDs, if so, we use the fields that have an ID. If there are no IDs at all and the PyArrow to Iceberg visitor returns an empty schema, then we introduce the name mapping (but maybe we should just leave that out of scope for 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] JonasJ-ap commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6997:
URL: https://github.com/apache/iceberg/pull/6997#discussion_r1133182512


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -356,14 +366,19 @@ def field(self, field: NestedField, field_result: pa.DataType) -> pa.Field:
             name=field.name,
             type=field_result,
             nullable=field.optional,
-            metadata={"doc": field.doc, "id": str(field.field_id)} if field.doc else {},
+            metadata={PYTHON_DOC.decode(): field.doc, PYTHON_FIELD_ID.decode(): str(field.field_id)}
+            if field.doc
+            else {PYTHON_FIELD_ID.decode(): str(field.field_id)},

Review Comment:
   Thank you very much for such a detailed explanation. I was thinking of the "pre-processing PyArrow schema" approach mentioned by Ryan. But now I see the problem of the prefix and I agree that we should leave name mapping for another PR. 
   
   I refactored the `_get_field_id_and_doc` to search for key ended with `id` such that both PARQUET and ORC could be handled. I also checked the AVRO format and found that the related key name is `field-id` so it also fits:
   ```python
   (base) ➜  user_id_bucket=2 avro-tools getschema 20230312_025043_00060_6msjy-bf8f6d3b-7e93-4f25-8a77-d0c183d6771a.avro
   23/03/11 21:55:31 WARN util.NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
   {
     "type" : "record",
     "name" : "table",
     "fields" : [ {
       "name" : "user_id",
       "type" : [ "null", "int" ],
       "default" : null,
       "field-id" : 1
     }
   ```
   The only thing I am not sure about is whether doing `key.decode().endswith(FIELD_ID)` here is too flexible? If so, may be I can change it to 
   ```python
   PYARROW_FIELD_ID_KEYS = ["PARQUET:field_id", "id"]
   
   def _get_field_id_and_doc(field: pa.Field) -> Tuple[Optional[int], Optional[str]]:
       field_id = None
       doc = None
   
       for key_name in PYARROW_FIELD_ID_KEYS:
              if id_str := field.metadata.get(key_name):
                  field_id = str(id_str.decode())
   ....
   ```
   
   May I ask what do you think about this?
   



-- 
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 #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -476,6 +483,202 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results = []
+    for i in range(len(schema.names)):
+        field = schema.field(i)
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(Schema | IcebergType): An instance of a Schema or an IcebergType
+        visitor (PyarrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    struct_results = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyarrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""
+
+    def after_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a list element."""
+
+    def before_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map key."""
+
+    def after_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map key."""
+
+    def before_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map value."""
+
+    def after_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map value."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[T]) -> Schema:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[T]) -> T:
+        """visit a struct"""
+
+    @abstractmethod
+    def list(self, list_type: pa.ListType, element_result: T) -> T:
+        """visit a list"""
+
+    @abstractmethod
+    def map(self, map_type: pa.MapType, key_result: T, value_result: T) -> T:
+        """visit a map"""
+
+    @abstractmethod
+    def primitive(self, primitive: pa.DataType) -> T:
+        """visit a primitive type"""
+
+
+def _get_field_id(field: pa.Field) -> int:
+    if field.metadata is not None:
+        field_metadata = {k.decode(): v.decode() for k, v in field.metadata.items()}
+        if field_id := field_metadata.get("PARQUET:field_id"):
+            return int(field_id)
+    raise ValueError(f"Field {field.name} does not have a field_id")
+
+
+class _ConvertToIceberg(PyarrowSchemaVisitor[IcebergType], ABC):
+    def schema(self, schema: pa.Schema, field_results: List[IcebergType]) -> Schema:
+        fields = []
+        for i in range(len(schema.names)):
+            field = schema.field(i)
+            field_id = _get_field_id(field)
+            field_type = field_results[i]
+            if field_type is not None:
+                fields.append(NestedField(field_id, field.name, field_type, required=not field.nullable))
+        return Schema(*fields)
+
+    def struct(self, struct: pa.StructType, field_results: List[IcebergType]) -> IcebergType:
+        fields = []
+        for i in range(struct.num_fields):
+            field = struct[i]
+            field_id = _get_field_id(field)
+            # may need to check doc strings
+            field_type = field_results[i]
+            if field_type is not None:
+                fields.append(NestedField(field_id, field.name, field_type, required=not field.nullable))
+        return StructType(*fields)
+
+    def list(self, list_type: pa.ListType, element_result: IcebergType) -> IcebergType:
+        element_field = list_type.value_field
+        element_id = _get_field_id(element_field)
+        if element_result is not None:
+            return ListType(element_id, element_result, element_required=not element_field.nullable)
+        raise ValueError(f"List type must have element field: {list_type}")
+
+    def map(self, map_type: pa.MapType, key_result: IcebergType, value_result: IcebergType) -> IcebergType:
+        key_field = map_type.key_field
+        key_id = _get_field_id(key_field)
+        value_field = map_type.item_field
+        value_id = _get_field_id(value_field)
+        if key_result is not None and value_result is not None:
+            return MapType(key_id, key_result, value_id, value_result, value_required=not value_field.nullable)
+        raise ValueError(f"Map type must have key and value fields: {map_type}")
+
+    def primitive(self, primitive: pa.DataType) -> IcebergType:
+        if pa.types.is_boolean(primitive):
+            return BooleanType()
+        elif pa.types.is_int32(primitive) or pa.types.is_uint32(primitive):
+            return IntegerType()
+        elif pa.types.is_int64(primitive) or pa.types.is_uint64(primitive):
+            return LongType()
+        elif pa.types.is_float32(primitive):
+            return FloatType()
+        elif pa.types.is_float64(primitive):
+            return DoubleType()
+        elif pa.types.is_decimal(primitive):
+            if isinstance(primitive, pa.Decimal256Type):
+                primitive = cast(pa.Decimal256Type, primitive)
+            else:
+                primitive = cast(pa.Decimal128Type, primitive)
+            return DecimalType(primitive.precision, primitive.scale)
+        elif pa.types.is_string(primitive):
+            return StringType()
+        elif pa.types.is_date(primitive):
+            return DateType()
+        elif pa.types.is_time(primitive):

Review Comment:
   > The read code operates using PyIceberg schemas. So that is going to convert PyArrow to PyIceberg, prune columns, and the convert back to request a projection from PyArrow. If there is a lossy conversion then that will cause a problem. For example if a time type is in millis but is converted without that information, then the requested type will be a timestamp in micros and that may cause a problem.
   
   Good point, I think we should have round-trip tests here, where we go from Iceberg to PyArrow, and back to Iceberg.
   
   > I assume that's going to read full nested structures because the only information it has is the top-level column name. If that happens, I'm concerned that we're reading leaf columns that we don't actually need to. I think projection will correctly ignore them, but it would be nice to avoid reading them.
   
   I agree, I'm would have to look into the details if we prune the fields of the nested fields properly. In the future I'd like to replace this with the custom evolution strategy: https://github.com/apache/arrow/issues/33972#issuecomment-1414589158



-- 
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 #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -484,6 +493,204 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results: List[Optional[IcebergType]] = []
+    for i, _ in enumerate(schema.names):
+        field = schema.field(i)
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(Schema | pa.DataType): An instance of a Schema or an IcebergType
+        visitor (PyArrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    struct_results: List[Optional[T]] = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyArrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""
+
+    def after_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a list element."""
+
+    def before_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map key."""
+
+    def after_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map key."""
+
+    def before_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map value."""
+
+    def after_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map value."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[Optional[T]]) -> Schema:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[Optional[T]]) -> T:
+        """visit a struct"""
+
+    @abstractmethod
+    def list(self, list_type: pa.ListType, element_result: Optional[T]) -> Optional[T]:
+        """visit a list"""
+
+    @abstractmethod
+    def map(self, map_type: pa.MapType, key_result: Optional[T], value_result: Optional[T]) -> Optional[T]:
+        """visit a map"""
+
+    @abstractmethod
+    def primitive(self, primitive: pa.DataType) -> T:
+        """visit a primitive type"""
+
+
+def _get_field_id_and_doc(field: pa.Field) -> Tuple[Optional[int], Optional[str]]:
+    field_metadata = {k.decode(): v.decode() for k, v in field.metadata.items()}
+    field_id = None
+    doc = field_metadata.get("PARQUET:doc", field_metadata.get("doc"))
+
+    if field_id_str := field_metadata.get("PARQUET:field_id", field_metadata.get("field_id")):
+        field_id = int(field_id_str)
+
+    return field_id, doc
+
+
+class _ConvertToIceberg(PyArrowSchemaVisitor[IcebergType], ABC):
+    def schema(self, schema: pa.Schema, field_results: List[Optional[IcebergType]]) -> Schema:
+        fields = []
+        for i, _ in enumerate(schema.names):

Review Comment:
   Thanks!



-- 
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 #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -120,6 +124,10 @@
 ONE_MEGABYTE = 1024 * 1024
 BUFFER_SIZE = "buffer-size"
 ICEBERG_SCHEMA = b"iceberg.schema"
+FIELD_ID = "id"

Review Comment:
   This should be `field_id`, right?
   ```suggestion
   FIELD_ID = "field_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] JonasJ-ap commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6997:
URL: https://github.com/apache/iceberg/pull/6997#discussion_r1133137339


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -484,6 +493,204 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results: List[Optional[IcebergType]] = []
+    for i, _ in enumerate(schema.names):
+        field = schema.field(i)
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(Schema | pa.DataType): An instance of a Schema or an IcebergType
+        visitor (PyArrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    struct_results: List[Optional[T]] = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyArrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""
+
+    def after_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a list element."""
+
+    def before_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map key."""
+
+    def after_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map key."""
+
+    def before_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map value."""
+
+    def after_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map value."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[Optional[T]]) -> Schema:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[Optional[T]]) -> T:
+        """visit a struct"""
+
+    @abstractmethod
+    def list(self, list_type: pa.ListType, element_result: Optional[T]) -> Optional[T]:
+        """visit a list"""
+
+    @abstractmethod
+    def map(self, map_type: pa.MapType, key_result: Optional[T], value_result: Optional[T]) -> Optional[T]:
+        """visit a map"""
+
+    @abstractmethod
+    def primitive(self, primitive: pa.DataType) -> T:
+        """visit a primitive type"""
+
+
+def _get_field_id_and_doc(field: pa.Field) -> Tuple[Optional[int], Optional[str]]:
+    field_metadata = {k.decode(): v.decode() for k, v in field.metadata.items()}
+    field_id = None
+    doc = field_metadata.get("PARQUET:doc", field_metadata.get("doc"))
+
+    if field_id_str := field_metadata.get("PARQUET:field_id", field_metadata.get("field_id")):

Review Comment:
   Thank you for your suggestions, I made constants for all the metadata names: `field_id`, `doc`, 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] JonasJ-ap commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6997:
URL: https://github.com/apache/iceberg/pull/6997#discussion_r1125303505


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -492,11 +710,7 @@ def _file_to_table(
         schema_raw = None
         if metadata := parquet_schema.metadata:
             schema_raw = metadata.get(ICEBERG_SCHEMA)
-        if schema_raw is None:
-            raise ValueError(
-                "Iceberg schema is not embedded into the Parquet file, see https://github.com/apache/iceberg/issues/6505"
-            )
-        file_schema = Schema.parse_raw(schema_raw)
+        file_schema = Schema.parse_raw(schema_raw) if schema_raw is not None else pyarrow_to_schema(parquet_schema)

Review Comment:
   Added a TODO comment here. Just curious, will the fallback level be something like the NameMapping feature?



-- 
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 merged pull request #6997: Python: Infer Iceberg schema from the Parquet file

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


-- 
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 #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -486,6 +499,207 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results: List[Optional[IcebergType]] = []
+    for field in schema:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(pa.DataType): An instance of a Schema or an IcebergType
+        visitor (PyArrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    struct_results: List[Optional[T]] = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got: {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyArrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""
+
+    def after_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a list element."""
+
+    def before_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map key."""
+
+    def after_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map key."""
+
+    def before_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map value."""
+
+    def after_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map value."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[Optional[T]]) -> Schema:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[Optional[T]]) -> T:
+        """visit a struct"""
+
+    @abstractmethod
+    def list(self, list_type: pa.ListType, element_result: Optional[T]) -> Optional[T]:
+        """visit a list"""
+
+    @abstractmethod
+    def map(self, map_type: pa.MapType, key_result: Optional[T], value_result: Optional[T]) -> Optional[T]:
+        """visit a map"""
+
+    @abstractmethod
+    def primitive(self, primitive: pa.DataType) -> T:
+        """visit a primitive type"""
+
+
+def _get_field_id_and_doc(field: pa.Field) -> Tuple[Optional[int], Optional[str]]:
+    field_id = None
+    doc = None
+
+    for pyarrow_field_id_key in PYARROW_FIELD_ID_KEYS:
+        if field_id_str := field.metadata.get(pyarrow_field_id_key):
+            field_id = int(field_id_str.decode())
+            break
+
+    for pyarrow_doc_key in PYARROW_FIELD_DOC_KEYS:
+        if doc_str := field.metadata.get(pyarrow_doc_key):
+            doc = doc_str.decode()
+            break
+
+    return field_id, doc
+
+
+class _ConvertToIceberg(PyArrowSchemaVisitor[IcebergType]):
+    def schema(self, schema: pa.Schema, field_results: List[Optional[IcebergType]]) -> Schema:
+        fields = []
+        for i, field in enumerate(schema):
+            field_id, field_doc = _get_field_id_and_doc(field)
+            field_type = field_results[i]
+            if field_type is not None and field_id is not None:
+                fields.append(NestedField(field_id, field.name, field_type, required=not field.nullable, doc=field_doc))
+        return Schema(*fields)
+
+    def struct(self, struct: pa.StructType, field_results: List[Optional[IcebergType]]) -> IcebergType:
+        fields = []
+        for i, field in enumerate(struct):
+            field_id, field_doc = _get_field_id_and_doc(field)
+            field_type = field_results[i]
+            if field_type is not None and field_id is not None:
+                fields.append(NestedField(field_id, field.name, field_type, required=not field.nullable, doc=field_doc))
+        return StructType(*fields)
+
+    def list(self, list_type: pa.ListType, element_result: Optional[IcebergType]) -> Optional[IcebergType]:
+        element_field = list_type.value_field
+        element_id, _ = _get_field_id_and_doc(element_field)
+        if element_result is not None and element_id is not None:
+            return ListType(element_id, element_result, element_required=not element_field.nullable)
+        return None
+
+    def map(
+        self, map_type: pa.MapType, key_result: Optional[IcebergType], value_result: Optional[IcebergType]
+    ) -> Optional[IcebergType]:
+        key_field = map_type.key_field
+        key_id, _ = _get_field_id_and_doc(key_field)
+        value_field = map_type.item_field
+        value_id, _ = _get_field_id_and_doc(value_field)

Review Comment:
   Why combine the lookup into a single method when doc is ignored most of the time?



-- 
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 #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -486,6 +499,207 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results: List[Optional[IcebergType]] = []
+    for field in schema:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)

Review Comment:
   Why isn't this done in the `visit_pyarrow` method? It seems incorrect to have top-level logic that must be copied into every function that uses a visitor.
   
   I'd expect something like this:



-- 
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 #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -486,6 +499,207 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results: List[Optional[IcebergType]] = []
+    for field in schema:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)

Review Comment:
   Why isn't this done in the `visit_pyarrow` method? It seems incorrect to have top-level logic that must be copied into every function that uses a visitor.



-- 
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] JonasJ-ap commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6997:
URL: https://github.com/apache/iceberg/pull/6997#discussion_r1179626777


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -507,11 +709,8 @@ def _file_to_table(
         schema_raw = None
         if metadata := physical_schema.metadata:
             schema_raw = metadata.get(ICEBERG_SCHEMA)
-        if schema_raw is None:
-            raise ValueError(
-                "Iceberg schema is not embedded into the Parquet file, see https://github.com/apache/iceberg/issues/6505"
-            )
-        file_schema = Schema.parse_raw(schema_raw)
+        # TODO: if field_ids are not present, Name Mapping should be implemented to look them up in the table schema

Review Comment:
   I created the issue: https://github.com/apache/iceberg/issues/7451.
   
   But I am not sure if what is the proper way to raise exception in this case. Based on my understanding, name mapping is also needed if portion of parquet fields miss the field ids. However, in this case, `pyarrow_to_schema` can still generate a valid iceberg schema for the the rest of parquet fields. It seems we should not raise exception in this case.
   
   Should we only raise exception when no field id exist in the data file? I think we can also log some warning messages when a pyarrow field containing a field id. What do you think?
   



-- 
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] JonasJ-ap commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6997:
URL: https://github.com/apache/iceberg/pull/6997#discussion_r1125299771


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -476,6 +483,217 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:

Review Comment:
   Thank you for your suggestions. I did not realize that `pa.DataType` can be used to to catch all the primitives. I refactored the visitor to take advantage of the `singledispatch`.



-- 
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 #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -492,11 +695,8 @@ def _file_to_table(
         schema_raw = None
         if metadata := parquet_schema.metadata:
             schema_raw = metadata.get(ICEBERG_SCHEMA)
-        if schema_raw is None:
-            raise ValueError(
-                "Iceberg schema is not embedded into the Parquet file, see https://github.com/apache/iceberg/issues/6505"
-            )
-        file_schema = Schema.parse_raw(schema_raw)
+        # TODO: if field_ids are not present, another fallback level should be implemented to look them up in the table schema

Review Comment:
   This is Iceberg's "name mapping" feature.
   
   The basics:
   1. Name mapping is applied ONLY if there are not field IDs in the Parquet file
   2. If a field is present in the name mapping, the ID from the mapping is used. Multiple names can be mapped to the same ID
   3. If a field is not present in the name mapping, it is omitted.
   
   For PyIceberg, the name mapping would probably be implemented by pre-processing the PyArrow schema to add the `PYTHON:field_id` properties. That way this code doesn't need to change.
   
   As a result, if there is no field ID for any field, it should be omitted from the Iceberg 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] JonasJ-ap commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6997:
URL: https://github.com/apache/iceberg/pull/6997#discussion_r1125299771


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -476,6 +483,217 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:

Review Comment:
   Thank you for your suggestions. I did not realize that I can use `pa.DataType` to catch all the primitives. I refactored the visitor to take advantage of the `singledispatch`.



-- 
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 #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -476,6 +483,217 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    return visit_arrow_schema(schema, _ConvertToIceberg())
+
+
+def visit_arrow_schema(obj: pa.Schema, visitor: ArrowSchemaVisitor[T]) -> Schema:
+    struct_results = []
+    for i in range(len(obj.names)):
+        field = obj.field(i)
+        visitor.before_field(field)
+        struct_result = visit_arrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.schema(obj, struct_results)
+
+
+def visit_arrow(obj: pa.DataType, visitor: ArrowSchemaVisitor[T]) -> T:
+    if pa.types.is_struct(obj):
+        return visit_arrow_struct(obj, visitor)
+    elif pa.types.is_list(obj):
+        return visit_arrow_list(obj, visitor)
+    elif pa.types.is_map(obj):
+        return visit_arrow_map(obj, visitor)
+    else:
+        return visit_arrow_primitive(obj, visitor)
+
+
+def visit_arrow_struct(obj: pa.DataType, visitor: ArrowSchemaVisitor[T]) -> T:
+    if not pa.types.is_struct(obj):
+        raise TypeError(f"Expected struct type, got {type(obj)}")
+    obj = cast(pa.StructType, obj)
+    struct_results = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_arrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+def visit_arrow_list(obj: pa.DataType, visitor: ArrowSchemaVisitor[T]) -> T:
+    if not pa.types.is_list(obj):
+        raise TypeError(f"Expected list type, got {type(obj)}")
+    obj = cast(pa.ListType, obj)
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_arrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+def visit_arrow_map(obj: pa.DataType, visitor: ArrowSchemaVisitor[T]) -> T:
+    if not pa.types.is_map(obj):
+        raise TypeError(f"Expected map type, got {type(obj)}")
+    obj = cast(pa.MapType, obj)
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_arrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_arrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+def visit_arrow_primitive(obj: pa.DataType, visitor: ArrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class ArrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""
+
+    def after_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a list element."""
+
+    def before_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map key."""
+
+    def after_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map key."""
+
+    def before_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map value."""
+
+    def after_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map value."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[T]) -> Schema:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[T]) -> T:
+        """visit a struct"""
+
+    @abstractmethod
+    def list(self, list_type: pa.ListType, element_result: T) -> T:
+        """visit a list"""
+
+    @abstractmethod
+    def map(self, map_type: pa.MapType, key_result: T, value_result: T) -> T:
+        """visit a map"""
+
+    @abstractmethod
+    def primitive(self, primitive: pa.DataType) -> T:
+        """visit a primitive type"""
+
+
+def _get_field_id(field: pa.Field) -> int:
+    field_metadata = {k.decode(): v.decode() for k, v in field.metadata.items()}
+    if field_id := field_metadata.get("PARQUET:field_id"):
+        return field_id

Review Comment:
   Would it be possible to extract the schema from the test that you did against Glue, and add that conversion as a unit test? I'm surprised that this works. I think that this is an `str` and not an `int`.



-- 
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] sheinbergon commented on pull request #6997: Python: Infer Iceberg schema from the Parquet file

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

   @Fokko @JonasJ-ap thank you for your efforts in resolving these issues. Any chance this PR will be merged soon, and included as part of 0.4.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 #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -476,6 +483,202 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results = []
+    for i in range(len(schema.names)):
+        field = schema.field(i)
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(Schema | IcebergType): An instance of a Schema or an IcebergType
+        visitor (PyarrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    struct_results = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyarrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""
+
+    def after_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a list element."""
+
+    def before_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map key."""
+
+    def after_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map key."""
+
+    def before_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map value."""
+
+    def after_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map value."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[T]) -> Schema:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[T]) -> T:
+        """visit a struct"""
+
+    @abstractmethod
+    def list(self, list_type: pa.ListType, element_result: T) -> T:
+        """visit a list"""
+
+    @abstractmethod
+    def map(self, map_type: pa.MapType, key_result: T, value_result: T) -> T:
+        """visit a map"""
+
+    @abstractmethod
+    def primitive(self, primitive: pa.DataType) -> T:
+        """visit a primitive type"""
+
+
+def _get_field_id(field: pa.Field) -> int:
+    if field.metadata is not None:
+        field_metadata = {k.decode(): v.decode() for k, v in field.metadata.items()}
+        if field_id := field_metadata.get("PARQUET:field_id"):
+            return int(field_id)
+    raise ValueError(f"Field {field.name} does not have a field_id")
+
+
+class _ConvertToIceberg(PyarrowSchemaVisitor[IcebergType], ABC):
+    def schema(self, schema: pa.Schema, field_results: List[IcebergType]) -> Schema:
+        fields = []
+        for i in range(len(schema.names)):
+            field = schema.field(i)
+            field_id = _get_field_id(field)
+            field_type = field_results[i]
+            if field_type is not None:
+                fields.append(NestedField(field_id, field.name, field_type, required=not field.nullable))
+        return Schema(*fields)
+
+    def struct(self, struct: pa.StructType, field_results: List[IcebergType]) -> IcebergType:
+        fields = []
+        for i in range(struct.num_fields):
+            field = struct[i]
+            field_id = _get_field_id(field)
+            # may need to check doc strings
+            field_type = field_results[i]
+            if field_type is not None:
+                fields.append(NestedField(field_id, field.name, field_type, required=not field.nullable))
+        return StructType(*fields)
+
+    def list(self, list_type: pa.ListType, element_result: IcebergType) -> IcebergType:
+        element_field = list_type.value_field
+        element_id = _get_field_id(element_field)
+        if element_result is not None:
+            return ListType(element_id, element_result, element_required=not element_field.nullable)
+        raise ValueError(f"List type must have element field: {list_type}")
+
+    def map(self, map_type: pa.MapType, key_result: IcebergType, value_result: IcebergType) -> IcebergType:
+        key_field = map_type.key_field
+        key_id = _get_field_id(key_field)
+        value_field = map_type.item_field
+        value_id = _get_field_id(value_field)
+        if key_result is not None and value_result is not None:
+            return MapType(key_id, key_result, value_id, value_result, value_required=not value_field.nullable)
+        raise ValueError(f"Map type must have key and value fields: {map_type}")
+
+    def primitive(self, primitive: pa.DataType) -> IcebergType:
+        if pa.types.is_boolean(primitive):
+            return BooleanType()
+        elif pa.types.is_int32(primitive) or pa.types.is_uint32(primitive):
+            return IntegerType()
+        elif pa.types.is_int64(primitive) or pa.types.is_uint64(primitive):
+            return LongType()
+        elif pa.types.is_float32(primitive):
+            return FloatType()
+        elif pa.types.is_float64(primitive):
+            return DoubleType()
+        elif pa.types.is_decimal(primitive):
+            if isinstance(primitive, pa.Decimal256Type):
+                primitive = cast(pa.Decimal256Type, primitive)
+            else:
+                primitive = cast(pa.Decimal128Type, primitive)
+            return DecimalType(primitive.precision, primitive.scale)
+        elif pa.types.is_string(primitive):
+            return StringType()
+        elif pa.types.is_date(primitive):
+            return DateType()
+        elif pa.types.is_time(primitive):
+            return TimeType()
+        elif pa.types.is_timestamp(primitive):
+            primitive = cast(pa.TimestampType, primitive)
+            if primitive.tz is not None:

Review Comment:
   @Fokko, can we handle non-UTC time zones, or should we fail if this is not UTC or +00:00?



-- 
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] JonasJ-ap commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6997:
URL: https://github.com/apache/iceberg/pull/6997#discussion_r1125300366


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -476,6 +483,217 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    return visit_arrow_schema(schema, _ConvertToIceberg())
+
+
+def visit_arrow_schema(obj: pa.Schema, visitor: ArrowSchemaVisitor[T]) -> Schema:
+    struct_results = []
+    for i in range(len(obj.names)):
+        field = obj.field(i)
+        visitor.before_field(field)
+        struct_result = visit_arrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.schema(obj, struct_results)
+
+
+def visit_arrow(obj: pa.DataType, visitor: ArrowSchemaVisitor[T]) -> T:
+    if pa.types.is_struct(obj):
+        return visit_arrow_struct(obj, visitor)
+    elif pa.types.is_list(obj):
+        return visit_arrow_list(obj, visitor)
+    elif pa.types.is_map(obj):
+        return visit_arrow_map(obj, visitor)
+    else:
+        return visit_arrow_primitive(obj, visitor)
+
+
+def visit_arrow_struct(obj: pa.DataType, visitor: ArrowSchemaVisitor[T]) -> T:
+    if not pa.types.is_struct(obj):
+        raise TypeError(f"Expected struct type, got {type(obj)}")
+    obj = cast(pa.StructType, obj)
+    struct_results = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_arrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+def visit_arrow_list(obj: pa.DataType, visitor: ArrowSchemaVisitor[T]) -> T:
+    if not pa.types.is_list(obj):
+        raise TypeError(f"Expected list type, got {type(obj)}")
+    obj = cast(pa.ListType, obj)
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_arrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+def visit_arrow_map(obj: pa.DataType, visitor: ArrowSchemaVisitor[T]) -> T:
+    if not pa.types.is_map(obj):
+        raise TypeError(f"Expected map type, got {type(obj)}")
+    obj = cast(pa.MapType, obj)
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_arrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_arrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+def visit_arrow_primitive(obj: pa.DataType, visitor: ArrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class ArrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""
+
+    def after_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a list element."""
+
+    def before_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map key."""
+
+    def after_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map key."""
+
+    def before_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map value."""
+
+    def after_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map value."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[T]) -> Schema:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[T]) -> T:
+        """visit a struct"""
+
+    @abstractmethod
+    def list(self, list_type: pa.ListType, element_result: T) -> T:
+        """visit a list"""
+
+    @abstractmethod
+    def map(self, map_type: pa.MapType, key_result: T, value_result: T) -> T:
+        """visit a map"""
+
+    @abstractmethod
+    def primitive(self, primitive: pa.DataType) -> T:
+        """visit a primitive type"""
+
+
+def _get_field_id(field: pa.Field) -> int:
+    field_metadata = {k.decode(): v.decode() for k, v in field.metadata.items()}
+    if field_id := field_metadata.get("PARQUET:field_id"):
+        return field_id
+    raise ValueError(f"Field {field.name} does not have a field_id")
+
+
+class _ConvertToIceberg(ArrowSchemaVisitor[IcebergType], ABC):
+    def schema(self, schema: pa.Schema, field_results: List[IcebergType]) -> Schema:
+        fields = []
+        for i in range(len(schema.names)):
+            field = schema.field(i)
+            field_id = _get_field_id(field)
+            field_type = field_results[i]
+            if field_id is not None and field_type is not None:
+                if field.nullable:
+                    fields.append(NestedField(field_id, field.name, field_type, False))
+                else:
+                    fields.append(NestedField(field_id, field.name, field_type, True))
+        return Schema(*fields)
+
+    def struct(self, struct: pa.StructType, field_results: List[IcebergType]) -> IcebergType:
+        fields = []
+        for i in range(struct.num_fields):
+            field = struct[i]
+            field_id = _get_field_id(field)
+            # may need to check doc strings
+            field_type = field_results[i]
+            if field_id is not None and field_type is not None:
+                if field.nullable:
+                    fields.append(NestedField(field_id, field.name, field_type, False))
+                else:
+                    fields.append(NestedField(field_id, field.name, field_type, True))
+        return StructType(*fields)
+
+    def list(self, list_type: pa.ListType, element_result: IcebergType) -> IcebergType:
+        element_field = list_type.value_field
+        element_id = _get_field_id(element_field)
+        if element_id is not None and element_result is not None:
+            if element_field.nullable:
+                return ListType(element_id, element_result, False)
+            else:
+                return ListType(element_id, element_result, True)
+        raise ValueError("List type must have element field")

Review Comment:
   Thank you for your suggestion. It will definitely make the exception message more clear!



-- 
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] JonasJ-ap commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6997:
URL: https://github.com/apache/iceberg/pull/6997#discussion_r1131154963


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -476,6 +483,202 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results = []
+    for i in range(len(schema.names)):
+        field = schema.field(i)
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(Schema | IcebergType): An instance of a Schema or an IcebergType
+        visitor (PyarrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    struct_results = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyarrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""
+
+    def after_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a list element."""
+
+    def before_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map key."""
+
+    def after_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map key."""
+
+    def before_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map value."""
+
+    def after_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map value."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[T]) -> Schema:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[T]) -> T:
+        """visit a struct"""
+
+    @abstractmethod
+    def list(self, list_type: pa.ListType, element_result: T) -> T:
+        """visit a list"""
+
+    @abstractmethod
+    def map(self, map_type: pa.MapType, key_result: T, value_result: T) -> T:
+        """visit a map"""
+
+    @abstractmethod
+    def primitive(self, primitive: pa.DataType) -> T:
+        """visit a primitive type"""
+
+
+def _get_field_id(field: pa.Field) -> int:
+    if field.metadata is not None:
+        field_metadata = {k.decode(): v.decode() for k, v in field.metadata.items()}
+        if field_id := field_metadata.get("PARQUET:field_id"):
+            return int(field_id)
+    raise ValueError(f"Field {field.name} does not have a field_id")
+
+
+class _ConvertToIceberg(PyarrowSchemaVisitor[IcebergType], ABC):
+    def schema(self, schema: pa.Schema, field_results: List[IcebergType]) -> Schema:
+        fields = []
+        for i in range(len(schema.names)):
+            field = schema.field(i)
+            field_id = _get_field_id(field)
+            field_type = field_results[i]
+            if field_type is not None:
+                fields.append(NestedField(field_id, field.name, field_type, required=not field.nullable))
+        return Schema(*fields)
+
+    def struct(self, struct: pa.StructType, field_results: List[IcebergType]) -> IcebergType:
+        fields = []
+        for i in range(struct.num_fields):
+            field = struct[i]
+            field_id = _get_field_id(field)
+            # may need to check doc strings

Review Comment:
   Thank you for your explanation. I added relevant code to handle the optional doc if exists.
   
   I did some tests and found that optional doc seemed not to be stored in parquet field metadata:
   The table is created by :
   ```python
   spark.sql("CREATE TABLE demo.iceberg_ref.schema_doc2 (id bigint COMMENT 'unique id', data string) USING iceberg")
   ```
   Fetch parquet schema:
   ```python
   doc_test_path = "/Users/jonasjiang/.CMVolumes/gluetestjonas/warehouse/iceberg_ref.db/schema_doc2/data/00000-0-49fb8801-9660-47ee-a8c7-b2af7fb703b4-00001.parquet"
   with fs.open_input_file(doc_test_path) as f:
       parquet_schema = pq.read_schema(f)
       print("=======parquet schema=======")
       print(parquet_schema)
       print("=======iceberg schema=======")
       print(Schema.parse_raw(parquet_schema.metadata[b'iceberg.schema'].decode()))
   ```
   Result:
   ```python
   =======parquet schema=======
   id: int64 not null
     -- field metadata --
     PARQUET:field_id: '1'
   data: string not null
     -- field metadata --
     PARQUET:field_id: '2'
   -- schema metadata --
   iceberg.schema: '{"type":"struct","schema-id":0,"fields":[{"id":1,"name":' + 110
   =======iceberg schema=======
   table {
     1: id: required long (unique id)
     2: data: required string
   }
   ```
   



-- 
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] JonasJ-ap commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6997:
URL: https://github.com/apache/iceberg/pull/6997#discussion_r1133128207


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -484,6 +493,204 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results: List[Optional[IcebergType]] = []
+    for i, _ in enumerate(schema.names):
+        field = schema.field(i)
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(Schema | pa.DataType): An instance of a Schema or an IcebergType
+        visitor (PyArrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    struct_results: List[Optional[T]] = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyArrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""
+
+    def after_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a list element."""
+
+    def before_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map key."""
+
+    def after_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map key."""
+
+    def before_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map value."""
+
+    def after_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map value."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[Optional[T]]) -> Schema:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[Optional[T]]) -> T:
+        """visit a struct"""
+
+    @abstractmethod
+    def list(self, list_type: pa.ListType, element_result: Optional[T]) -> Optional[T]:
+        """visit a list"""
+
+    @abstractmethod
+    def map(self, map_type: pa.MapType, key_result: Optional[T], value_result: Optional[T]) -> Optional[T]:
+        """visit a map"""
+
+    @abstractmethod
+    def primitive(self, primitive: pa.DataType) -> T:
+        """visit a primitive type"""
+
+
+def _get_field_id_and_doc(field: pa.Field) -> Tuple[Optional[int], Optional[str]]:
+    field_metadata = {k.decode(): v.decode() for k, v in field.metadata.items()}
+    field_id = None
+    doc = field_metadata.get("PARQUET:doc", field_metadata.get("doc"))
+
+    if field_id_str := field_metadata.get("PARQUET:field_id", field_metadata.get("field_id")):
+        field_id = int(field_id_str)
+
+    return field_id, doc
+
+
+class _ConvertToIceberg(PyArrowSchemaVisitor[IcebergType], ABC):
+    def schema(self, schema: pa.Schema, field_results: List[Optional[IcebergType]]) -> Schema:
+        fields = []
+        for i, _ in enumerate(schema.names):

Review Comment:
   It seems `fields` is not an arrribute of pyarrow schema in `pyarrow v11.0.0`😢. I got the following error:
   ```python
   AttributeError: 'pyarrow.lib.Schema' object has no attribute 'fields'
   ```
   
   ref: https://arrow.apache.org/docs/python/generated/pyarrow.Schema.html#pyarrow.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] JonasJ-ap commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6997:
URL: https://github.com/apache/iceberg/pull/6997#discussion_r1131185611


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -476,6 +483,202 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results = []
+    for i in range(len(schema.names)):
+        field = schema.field(i)
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(Schema | IcebergType): An instance of a Schema or an IcebergType
+        visitor (PyarrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    struct_results = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyarrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""
+
+    def after_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a list element."""
+
+    def before_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map key."""
+
+    def after_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map key."""
+
+    def before_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map value."""
+
+    def after_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map value."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[T]) -> Schema:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[T]) -> T:
+        """visit a struct"""
+
+    @abstractmethod
+    def list(self, list_type: pa.ListType, element_result: T) -> T:
+        """visit a list"""
+
+    @abstractmethod
+    def map(self, map_type: pa.MapType, key_result: T, value_result: T) -> T:
+        """visit a map"""
+
+    @abstractmethod
+    def primitive(self, primitive: pa.DataType) -> T:
+        """visit a primitive type"""
+
+
+def _get_field_id(field: pa.Field) -> int:
+    if field.metadata is not None:
+        field_metadata = {k.decode(): v.decode() for k, v in field.metadata.items()}
+        if field_id := field_metadata.get("PARQUET:field_id"):
+            return int(field_id)
+    raise ValueError(f"Field {field.name} does not have a field_id")
+
+
+class _ConvertToIceberg(PyarrowSchemaVisitor[IcebergType], ABC):
+    def schema(self, schema: pa.Schema, field_results: List[IcebergType]) -> Schema:
+        fields = []
+        for i in range(len(schema.names)):
+            field = schema.field(i)
+            field_id = _get_field_id(field)
+            field_type = field_results[i]
+            if field_type is not None:
+                fields.append(NestedField(field_id, field.name, field_type, required=not field.nullable))
+        return Schema(*fields)
+
+    def struct(self, struct: pa.StructType, field_results: List[IcebergType]) -> IcebergType:
+        fields = []
+        for i in range(struct.num_fields):
+            field = struct[i]
+            field_id = _get_field_id(field)
+            # may need to check doc strings
+            field_type = field_results[i]
+            if field_type is not None:
+                fields.append(NestedField(field_id, field.name, field_type, required=not field.nullable))
+        return StructType(*fields)
+
+    def list(self, list_type: pa.ListType, element_result: IcebergType) -> IcebergType:
+        element_field = list_type.value_field
+        element_id = _get_field_id(element_field)
+        if element_result is not None:

Review Comment:
   Thanks for you clarification. I refactored the visitor to accommodate the change.



-- 
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 #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -484,6 +495,203 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results: List[Optional[IcebergType]] = []
+    for field in schema:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(pa.DataType): An instance of a Schema or an IcebergType
+        visitor (PyArrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    struct_results: List[Optional[T]] = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyArrowSchemaVisitor(Generic[T], ABC):

Review Comment:
   Minor, this is a concrete class
   ```suggestion
   class PyArrowSchemaVisitor(Generic[T]):
   ```



-- 
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 #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -122,6 +126,12 @@
 ONE_MEGABYTE = 1024 * 1024
 BUFFER_SIZE = "buffer-size"
 ICEBERG_SCHEMA = b"iceberg.schema"
+FIELD_ID = "field_id"
+FIELD_DOC = "doc"
+PYARROW_FIELD_ID_KEYS = [b"PARQUET:field_id", b"field_id"]

Review Comment:
   Sounds reasonable.



-- 
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 #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -486,6 +499,207 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results: List[Optional[IcebergType]] = []
+    for field in schema:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(pa.DataType): An instance of a Schema or an IcebergType
+        visitor (PyArrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    struct_results: List[Optional[T]] = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got: {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyArrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""

Review Comment:
   In other visitors, the callbacks for list elements, map keys, and map values call `before_field` and `after_field` so that you can use just one if needed.



-- 
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 #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -122,6 +126,12 @@
 ONE_MEGABYTE = 1024 * 1024
 BUFFER_SIZE = "buffer-size"
 ICEBERG_SCHEMA = b"iceberg.schema"
+FIELD_ID = "field_id"
+FIELD_DOC = "doc"
+PYARROW_FIELD_ID_KEYS = [b"PARQUET:field_id", b"field_id"]

Review Comment:
   Why does this support `field_id` without a prefix?



-- 
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] JonasJ-ap commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6997:
URL: https://github.com/apache/iceberg/pull/6997#discussion_r1174072713


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -486,6 +499,207 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results: List[Optional[IcebergType]] = []
+    for field in schema:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(pa.DataType): An instance of a Schema or an IcebergType
+        visitor (PyArrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    struct_results: List[Optional[T]] = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got: {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyArrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""
+
+    def after_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a list element."""
+
+    def before_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map key."""
+
+    def after_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map key."""
+
+    def before_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map value."""
+
+    def after_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map value."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[Optional[T]]) -> Schema:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[Optional[T]]) -> T:
+        """visit a struct"""
+
+    @abstractmethod
+    def list(self, list_type: pa.ListType, element_result: Optional[T]) -> Optional[T]:
+        """visit a list"""
+
+    @abstractmethod
+    def map(self, map_type: pa.MapType, key_result: Optional[T], value_result: Optional[T]) -> Optional[T]:
+        """visit a map"""
+
+    @abstractmethod
+    def primitive(self, primitive: pa.DataType) -> T:
+        """visit a primitive type"""
+
+
+def _get_field_id_and_doc(field: pa.Field) -> Tuple[Optional[int], Optional[str]]:
+    field_id = None
+    doc = None
+
+    for pyarrow_field_id_key in PYARROW_FIELD_ID_KEYS:
+        if field_id_str := field.metadata.get(pyarrow_field_id_key):
+            field_id = int(field_id_str.decode())
+            break
+
+    for pyarrow_doc_key in PYARROW_FIELD_DOC_KEYS:
+        if doc_str := field.metadata.get(pyarrow_doc_key):
+            doc = doc_str.decode()
+            break
+
+    return field_id, doc
+
+
+class _ConvertToIceberg(PyArrowSchemaVisitor[IcebergType]):
+    def schema(self, schema: pa.Schema, field_results: List[Optional[IcebergType]]) -> Schema:
+        fields = []
+        for i, field in enumerate(schema):
+            field_id, field_doc = _get_field_id_and_doc(field)
+            field_type = field_results[i]
+            if field_type is not None and field_id is not None:
+                fields.append(NestedField(field_id, field.name, field_type, required=not field.nullable, doc=field_doc))
+        return Schema(*fields)
+
+    def struct(self, struct: pa.StructType, field_results: List[Optional[IcebergType]]) -> IcebergType:
+        fields = []
+        for i, field in enumerate(struct):
+            field_id, field_doc = _get_field_id_and_doc(field)
+            field_type = field_results[i]
+            if field_type is not None and field_id is not None:
+                fields.append(NestedField(field_id, field.name, field_type, required=not field.nullable, doc=field_doc))
+        return StructType(*fields)
+
+    def list(self, list_type: pa.ListType, element_result: Optional[IcebergType]) -> Optional[IcebergType]:
+        element_field = list_type.value_field
+        element_id, _ = _get_field_id_and_doc(element_field)
+        if element_result is not None and element_id is not None:
+            return ListType(element_id, element_result, element_required=not element_field.nullable)
+        return None
+
+    def map(
+        self, map_type: pa.MapType, key_result: Optional[IcebergType], value_result: Optional[IcebergType]
+    ) -> Optional[IcebergType]:
+        key_field = map_type.key_field
+        key_id, _ = _get_field_id_and_doc(key_field)
+        value_field = map_type.item_field
+        value_id, _ = _get_field_id_and_doc(value_field)

Review Comment:
   I separated it into `_get_field_id` and `_get_field_doc`



-- 
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] JonasJ-ap commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6997:
URL: https://github.com/apache/iceberg/pull/6997#discussion_r1174199419


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -476,6 +483,202 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results = []
+    for i in range(len(schema.names)):
+        field = schema.field(i)
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(Schema | IcebergType): An instance of a Schema or an IcebergType
+        visitor (PyarrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    struct_results = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyarrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""
+
+    def after_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a list element."""
+
+    def before_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map key."""
+
+    def after_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map key."""
+
+    def before_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map value."""
+
+    def after_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map value."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[T]) -> Schema:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[T]) -> T:
+        """visit a struct"""
+
+    @abstractmethod
+    def list(self, list_type: pa.ListType, element_result: T) -> T:
+        """visit a list"""
+
+    @abstractmethod
+    def map(self, map_type: pa.MapType, key_result: T, value_result: T) -> T:
+        """visit a map"""
+
+    @abstractmethod
+    def primitive(self, primitive: pa.DataType) -> T:
+        """visit a primitive type"""
+
+
+def _get_field_id(field: pa.Field) -> int:
+    if field.metadata is not None:
+        field_metadata = {k.decode(): v.decode() for k, v in field.metadata.items()}
+        if field_id := field_metadata.get("PARQUET:field_id"):
+            return int(field_id)
+    raise ValueError(f"Field {field.name} does not have a field_id")
+
+
+class _ConvertToIceberg(PyarrowSchemaVisitor[IcebergType], ABC):
+    def schema(self, schema: pa.Schema, field_results: List[IcebergType]) -> Schema:
+        fields = []
+        for i in range(len(schema.names)):
+            field = schema.field(i)
+            field_id = _get_field_id(field)
+            field_type = field_results[i]
+            if field_type is not None:
+                fields.append(NestedField(field_id, field.name, field_type, required=not field.nullable))
+        return Schema(*fields)
+
+    def struct(self, struct: pa.StructType, field_results: List[IcebergType]) -> IcebergType:
+        fields = []
+        for i in range(struct.num_fields):
+            field = struct[i]
+            field_id = _get_field_id(field)
+            # may need to check doc strings
+            field_type = field_results[i]
+            if field_type is not None:
+                fields.append(NestedField(field_id, field.name, field_type, required=not field.nullable))
+        return StructType(*fields)
+
+    def list(self, list_type: pa.ListType, element_result: IcebergType) -> IcebergType:
+        element_field = list_type.value_field
+        element_id = _get_field_id(element_field)
+        if element_result is not None:
+            return ListType(element_id, element_result, element_required=not element_field.nullable)
+        raise ValueError(f"List type must have element field: {list_type}")
+
+    def map(self, map_type: pa.MapType, key_result: IcebergType, value_result: IcebergType) -> IcebergType:
+        key_field = map_type.key_field
+        key_id = _get_field_id(key_field)
+        value_field = map_type.item_field
+        value_id = _get_field_id(value_field)
+        if key_result is not None and value_result is not None:
+            return MapType(key_id, key_result, value_id, value_result, value_required=not value_field.nullable)
+        raise ValueError(f"Map type must have key and value fields: {map_type}")
+
+    def primitive(self, primitive: pa.DataType) -> IcebergType:
+        if pa.types.is_boolean(primitive):
+            return BooleanType()
+        elif pa.types.is_int32(primitive) or pa.types.is_uint32(primitive):
+            return IntegerType()
+        elif pa.types.is_int64(primitive) or pa.types.is_uint64(primitive):
+            return LongType()
+        elif pa.types.is_float32(primitive):
+            return FloatType()
+        elif pa.types.is_float64(primitive):
+            return DoubleType()
+        elif pa.types.is_decimal(primitive):
+            if isinstance(primitive, pa.Decimal256Type):
+                primitive = cast(pa.Decimal256Type, primitive)
+            else:
+                primitive = cast(pa.Decimal128Type, primitive)
+            return DecimalType(primitive.precision, primitive.scale)
+        elif pa.types.is_string(primitive):
+            return StringType()
+        elif pa.types.is_date(primitive):
+            return DateType()
+        elif pa.types.is_time(primitive):

Review Comment:
   Thank you for your review and for bringing up your concerns. I'd like to understand better what you find suspicious about PyArrow's ability to read Python files with different time or timestamp representations.
   
   From what I understand, Iceberg's TimeType, TimestampType, and TimestampzType require `us` and `UTC`, and my current conversion ensures that no data is lost.
   
   In this case, PyArrow can support reading non-UTC timezones and `s`, `ms`, and `us` precision, but it does not support nanosecond precision since the final requested type during projection will be `us` and `UTC`.: https://github.com/apache/iceberg/blob/283107d3608e65804314ba8fd2cb76aafdc36bbd/python/pyiceberg/io/pyarrow.py#L397-L404
   
   I chose to restrict the precision to `us` and the timezone to `UTC` because the Iceberg specification requires all stored time/timestamp to be in this precision and timezone. Since the `pyarrow_to_schema` visitor is used to read an Iceberg table's data file, I believe we should only support `us` and `UTC`.
   
   However, I am also not very sure about it. Regarding support for other precision and timezone here, I think more discussion and modifications may be needed if we want to add other support. How about creating another PR if needed to address these concerns?
   
   Thank you again for your feedback, and please let me know if you have any further questions or 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] Fokko commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -476,6 +483,217 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    return visit_arrow_schema(schema, _ConvertToIceberg())
+
+
+def visit_arrow_schema(obj: pa.Schema, visitor: ArrowSchemaVisitor[T]) -> Schema:
+    struct_results = []
+    for i in range(len(obj.names)):
+        field = obj.field(i)
+        visitor.before_field(field)
+        struct_result = visit_arrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.schema(obj, struct_results)
+
+
+def visit_arrow(obj: pa.DataType, visitor: ArrowSchemaVisitor[T]) -> T:
+    if pa.types.is_struct(obj):
+        return visit_arrow_struct(obj, visitor)
+    elif pa.types.is_list(obj):
+        return visit_arrow_list(obj, visitor)
+    elif pa.types.is_map(obj):
+        return visit_arrow_map(obj, visitor)
+    else:
+        return visit_arrow_primitive(obj, visitor)
+
+
+def visit_arrow_struct(obj: pa.DataType, visitor: ArrowSchemaVisitor[T]) -> T:
+    if not pa.types.is_struct(obj):
+        raise TypeError(f"Expected struct type, got {type(obj)}")
+    obj = cast(pa.StructType, obj)
+    struct_results = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_arrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+def visit_arrow_list(obj: pa.DataType, visitor: ArrowSchemaVisitor[T]) -> T:
+    if not pa.types.is_list(obj):
+        raise TypeError(f"Expected list type, got {type(obj)}")
+    obj = cast(pa.ListType, obj)
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_arrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+def visit_arrow_map(obj: pa.DataType, visitor: ArrowSchemaVisitor[T]) -> T:
+    if not pa.types.is_map(obj):
+        raise TypeError(f"Expected map type, got {type(obj)}")
+    obj = cast(pa.MapType, obj)
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_arrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_arrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+def visit_arrow_primitive(obj: pa.DataType, visitor: ArrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class ArrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""
+
+    def after_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a list element."""
+
+    def before_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map key."""
+
+    def after_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map key."""
+
+    def before_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map value."""
+
+    def after_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map value."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[T]) -> Schema:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[T]) -> T:
+        """visit a struct"""
+
+    @abstractmethod
+    def list(self, list_type: pa.ListType, element_result: T) -> T:
+        """visit a list"""
+
+    @abstractmethod
+    def map(self, map_type: pa.MapType, key_result: T, value_result: T) -> T:
+        """visit a map"""
+
+    @abstractmethod
+    def primitive(self, primitive: pa.DataType) -> T:
+        """visit a primitive type"""
+
+
+def _get_field_id(field: pa.Field) -> int:
+    field_metadata = {k.decode(): v.decode() for k, v in field.metadata.items()}
+    if field_id := field_metadata.get("PARQUET:field_id"):
+        return field_id
+    raise ValueError(f"Field {field.name} does not have a field_id")
+
+
+class _ConvertToIceberg(ArrowSchemaVisitor[IcebergType], ABC):
+    def schema(self, schema: pa.Schema, field_results: List[IcebergType]) -> Schema:
+        fields = []
+        for i in range(len(schema.names)):
+            field = schema.field(i)
+            field_id = _get_field_id(field)
+            field_type = field_results[i]
+            if field_id is not None and field_type is not None:
+                if field.nullable:
+                    fields.append(NestedField(field_id, field.name, field_type, False))

Review Comment:
   How about inlining this `if`, and adding a named argument:
   ```suggestion
                       fields.append(NestedField(field_id, field.name, field_type, required=not field.nullable))
   ```



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

To unsubscribe, e-mail: 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 #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -476,6 +483,202 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results = []
+    for i in range(len(schema.names)):
+        field = schema.field(i)
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(Schema | IcebergType): An instance of a Schema or an IcebergType
+        visitor (PyarrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    struct_results = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyarrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""
+
+    def after_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a list element."""
+
+    def before_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map key."""
+
+    def after_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map key."""
+
+    def before_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map value."""
+
+    def after_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map value."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[T]) -> Schema:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[T]) -> T:
+        """visit a struct"""
+
+    @abstractmethod
+    def list(self, list_type: pa.ListType, element_result: T) -> T:
+        """visit a list"""
+
+    @abstractmethod
+    def map(self, map_type: pa.MapType, key_result: T, value_result: T) -> T:
+        """visit a map"""
+
+    @abstractmethod
+    def primitive(self, primitive: pa.DataType) -> T:
+        """visit a primitive type"""
+
+
+def _get_field_id(field: pa.Field) -> int:
+    if field.metadata is not None:
+        field_metadata = {k.decode(): v.decode() for k, v in field.metadata.items()}
+        if field_id := field_metadata.get("PARQUET:field_id"):
+            return int(field_id)
+    raise ValueError(f"Field {field.name} does not have a field_id")
+
+
+class _ConvertToIceberg(PyarrowSchemaVisitor[IcebergType], ABC):
+    def schema(self, schema: pa.Schema, field_results: List[IcebergType]) -> Schema:
+        fields = []
+        for i in range(len(schema.names)):
+            field = schema.field(i)
+            field_id = _get_field_id(field)
+            field_type = field_results[i]
+            if field_type is not None:
+                fields.append(NestedField(field_id, field.name, field_type, required=not field.nullable))
+        return Schema(*fields)
+
+    def struct(self, struct: pa.StructType, field_results: List[IcebergType]) -> IcebergType:
+        fields = []
+        for i in range(struct.num_fields):
+            field = struct[i]
+            field_id = _get_field_id(field)
+            # may need to check doc strings
+            field_type = field_results[i]
+            if field_type is not None:
+                fields.append(NestedField(field_id, field.name, field_type, required=not field.nullable))
+        return StructType(*fields)
+
+    def list(self, list_type: pa.ListType, element_result: IcebergType) -> IcebergType:
+        element_field = list_type.value_field
+        element_id = _get_field_id(element_field)
+        if element_result is not None:

Review Comment:
   This should ignore the type if the field ID is not present.



-- 
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 #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -484,6 +493,204 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results: List[Optional[IcebergType]] = []
+    for i, _ in enumerate(schema.names):
+        field = schema.field(i)
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(Schema | pa.DataType): An instance of a Schema or an IcebergType
+        visitor (PyArrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    struct_results: List[Optional[T]] = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyArrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""
+
+    def after_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a list element."""
+
+    def before_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map key."""
+
+    def after_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map key."""
+
+    def before_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map value."""
+
+    def after_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map value."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[Optional[T]]) -> Schema:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[Optional[T]]) -> T:
+        """visit a struct"""
+
+    @abstractmethod
+    def list(self, list_type: pa.ListType, element_result: Optional[T]) -> Optional[T]:
+        """visit a list"""
+
+    @abstractmethod
+    def map(self, map_type: pa.MapType, key_result: Optional[T], value_result: Optional[T]) -> Optional[T]:
+        """visit a map"""
+
+    @abstractmethod
+    def primitive(self, primitive: pa.DataType) -> T:
+        """visit a primitive type"""
+
+
+def _get_field_id_and_doc(field: pa.Field) -> Tuple[Optional[int], Optional[str]]:
+    field_metadata = {k.decode(): v.decode() for k, v in field.metadata.items()}
+    field_id = None
+    doc = field_metadata.get("PARQUET:doc", field_metadata.get("doc"))
+
+    if field_id_str := field_metadata.get("PARQUET:field_id", field_metadata.get("field_id")):
+        field_id = int(field_id_str)
+
+    return field_id, doc
+
+
+class _ConvertToIceberg(PyArrowSchemaVisitor[IcebergType], ABC):
+    def schema(self, schema: pa.Schema, field_results: List[Optional[IcebergType]]) -> Schema:
+        fields = []
+        for i, _ in enumerate(schema.names):

Review Comment:
   @JonasJ-ap I'm sorry. I tested this locally before posting, but somewhere I messed up.
   ```suggestion
           for i, field in enumerate(schema):
   ```
   Works locally :)



-- 
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] JonasJ-ap commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6997:
URL: https://github.com/apache/iceberg/pull/6997#discussion_r1131224348


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -476,6 +483,202 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results = []
+    for i in range(len(schema.names)):
+        field = schema.field(i)
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(Schema | IcebergType): An instance of a Schema or an IcebergType
+        visitor (PyarrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    struct_results = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyarrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""
+
+    def after_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a list element."""
+
+    def before_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map key."""
+
+    def after_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map key."""
+
+    def before_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map value."""
+
+    def after_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map value."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[T]) -> Schema:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[T]) -> T:
+        """visit a struct"""
+
+    @abstractmethod
+    def list(self, list_type: pa.ListType, element_result: T) -> T:
+        """visit a list"""
+
+    @abstractmethod
+    def map(self, map_type: pa.MapType, key_result: T, value_result: T) -> T:
+        """visit a map"""
+
+    @abstractmethod
+    def primitive(self, primitive: pa.DataType) -> T:
+        """visit a primitive type"""
+
+
+def _get_field_id(field: pa.Field) -> int:
+    if field.metadata is not None:
+        field_metadata = {k.decode(): v.decode() for k, v in field.metadata.items()}
+        if field_id := field_metadata.get("PARQUET:field_id"):
+            return int(field_id)
+    raise ValueError(f"Field {field.name} does not have a field_id")
+
+
+class _ConvertToIceberg(PyarrowSchemaVisitor[IcebergType], ABC):
+    def schema(self, schema: pa.Schema, field_results: List[IcebergType]) -> Schema:
+        fields = []
+        for i in range(len(schema.names)):
+            field = schema.field(i)
+            field_id = _get_field_id(field)
+            field_type = field_results[i]
+            if field_type is not None:
+                fields.append(NestedField(field_id, field.name, field_type, required=not field.nullable))
+        return Schema(*fields)
+
+    def struct(self, struct: pa.StructType, field_results: List[IcebergType]) -> IcebergType:
+        fields = []
+        for i in range(struct.num_fields):
+            field = struct[i]
+            field_id = _get_field_id(field)
+            # may need to check doc strings
+            field_type = field_results[i]
+            if field_type is not None:
+                fields.append(NestedField(field_id, field.name, field_type, required=not field.nullable))
+        return StructType(*fields)
+
+    def list(self, list_type: pa.ListType, element_result: IcebergType) -> IcebergType:
+        element_field = list_type.value_field
+        element_id = _get_field_id(element_field)
+        if element_result is not None:
+            return ListType(element_id, element_result, element_required=not element_field.nullable)
+        raise ValueError(f"List type must have element field: {list_type}")
+
+    def map(self, map_type: pa.MapType, key_result: IcebergType, value_result: IcebergType) -> IcebergType:
+        key_field = map_type.key_field
+        key_id = _get_field_id(key_field)
+        value_field = map_type.item_field
+        value_id = _get_field_id(value_field)
+        if key_result is not None and value_result is not None:
+            return MapType(key_id, key_result, value_id, value_result, value_required=not value_field.nullable)
+        raise ValueError(f"Map type must have key and value fields: {map_type}")
+
+    def primitive(self, primitive: pa.DataType) -> IcebergType:
+        if pa.types.is_boolean(primitive):
+            return BooleanType()
+        elif pa.types.is_int32(primitive) or pa.types.is_uint32(primitive):
+            return IntegerType()
+        elif pa.types.is_int64(primitive) or pa.types.is_uint64(primitive):
+            return LongType()
+        elif pa.types.is_float32(primitive):
+            return FloatType()
+        elif pa.types.is_float64(primitive):
+            return DoubleType()
+        elif pa.types.is_decimal(primitive):
+            if isinstance(primitive, pa.Decimal256Type):
+                primitive = cast(pa.Decimal256Type, primitive)
+            else:
+                primitive = cast(pa.Decimal128Type, primitive)
+            return DecimalType(primitive.precision, primitive.scale)
+        elif pa.types.is_string(primitive):
+            return StringType()
+        elif pa.types.is_date(primitive):
+            return DateType()
+        elif pa.types.is_time(primitive):

Review Comment:
   Thank you for your explanation. I refactored the code to make it consistent with the primitive conversion in iceberg to pyarrow conversion. Currently, It only allow UTC as timezone and us as unit.
   
   I also added some round trip tests for these 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 #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -484,6 +493,204 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results: List[Optional[IcebergType]] = []
+    for i, _ in enumerate(schema.names):
+        field = schema.field(i)
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(Schema | pa.DataType): An instance of a Schema or an IcebergType
+        visitor (PyArrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    struct_results: List[Optional[T]] = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyArrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""
+
+    def after_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a list element."""
+
+    def before_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map key."""
+
+    def after_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map key."""
+
+    def before_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map value."""
+
+    def after_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map value."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[Optional[T]]) -> Schema:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[Optional[T]]) -> T:
+        """visit a struct"""
+
+    @abstractmethod
+    def list(self, list_type: pa.ListType, element_result: Optional[T]) -> Optional[T]:
+        """visit a list"""
+
+    @abstractmethod
+    def map(self, map_type: pa.MapType, key_result: Optional[T], value_result: Optional[T]) -> Optional[T]:
+        """visit a map"""
+
+    @abstractmethod
+    def primitive(self, primitive: pa.DataType) -> T:
+        """visit a primitive type"""
+
+
+def _get_field_id_and_doc(field: pa.Field) -> Tuple[Optional[int], Optional[str]]:
+    field_metadata = {k.decode(): v.decode() for k, v in field.metadata.items()}
+    field_id = None
+    doc = field_metadata.get("PARQUET:doc", field_metadata.get("doc"))
+
+    if field_id_str := field_metadata.get("PARQUET:field_id", field_metadata.get("field_id")):

Review Comment:
   Nit: I would move `"PARQUET:field_id"` into a constant



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -484,6 +493,204 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results: List[Optional[IcebergType]] = []
+    for i, _ in enumerate(schema.names):
+        field = schema.field(i)
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(Schema | pa.DataType): An instance of a Schema or an IcebergType
+        visitor (PyArrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    struct_results: List[Optional[T]] = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyArrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""
+
+    def after_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a list element."""
+
+    def before_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map key."""
+
+    def after_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map key."""
+
+    def before_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map value."""
+
+    def after_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map value."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[Optional[T]]) -> Schema:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[Optional[T]]) -> T:
+        """visit a struct"""
+
+    @abstractmethod
+    def list(self, list_type: pa.ListType, element_result: Optional[T]) -> Optional[T]:
+        """visit a list"""
+
+    @abstractmethod
+    def map(self, map_type: pa.MapType, key_result: Optional[T], value_result: Optional[T]) -> Optional[T]:
+        """visit a map"""
+
+    @abstractmethod
+    def primitive(self, primitive: pa.DataType) -> T:
+        """visit a primitive type"""
+
+
+def _get_field_id_and_doc(field: pa.Field) -> Tuple[Optional[int], Optional[str]]:
+    field_metadata = {k.decode(): v.decode() for k, v in field.metadata.items()}
+    field_id = None
+    doc = field_metadata.get("PARQUET:doc", field_metadata.get("doc"))
+
+    if field_id_str := field_metadata.get("PARQUET:field_id", field_metadata.get("field_id")):
+        field_id = int(field_id_str)
+
+    return field_id, doc
+
+
+class _ConvertToIceberg(PyArrowSchemaVisitor[IcebergType], ABC):
+    def schema(self, schema: pa.Schema, field_results: List[Optional[IcebergType]]) -> Schema:
+        fields = []
+        for i, _ in enumerate(schema.names):

Review Comment:
   ```suggestion
           for i, field in enumerate(schema.fields):
   ```
   And then we can lose the lookup on the next line.



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -576,7 +780,8 @@ def project_table(
             for table in pool.starmap(
                 func=_file_to_table,
                 iterable=[(fs, task, bound_row_filter, projected_schema, projected_field_ids, case_sensitive) for task in tasks],
-                chunksize=None,  # we could use this to control how to materialize the generator of tasks (we should also make the expression above lazy)
+                chunksize=None,

Review Comment:
   Unrelated change



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -484,6 +493,204 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results: List[Optional[IcebergType]] = []
+    for i, _ in enumerate(schema.names):
+        field = schema.field(i)
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(Schema | pa.DataType): An instance of a Schema or an IcebergType
+        visitor (PyArrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    struct_results: List[Optional[T]] = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyArrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""
+
+    def after_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a list element."""
+
+    def before_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map key."""
+
+    def after_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map key."""
+
+    def before_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map value."""
+
+    def after_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map value."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[Optional[T]]) -> Schema:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[Optional[T]]) -> T:
+        """visit a struct"""
+
+    @abstractmethod
+    def list(self, list_type: pa.ListType, element_result: Optional[T]) -> Optional[T]:
+        """visit a list"""
+
+    @abstractmethod
+    def map(self, map_type: pa.MapType, key_result: Optional[T], value_result: Optional[T]) -> Optional[T]:
+        """visit a map"""
+
+    @abstractmethod
+    def primitive(self, primitive: pa.DataType) -> T:
+        """visit a primitive type"""
+
+
+def _get_field_id_and_doc(field: pa.Field) -> Tuple[Optional[int], Optional[str]]:
+    field_metadata = {k.decode(): v.decode() for k, v in field.metadata.items()}
+    field_id = None
+    doc = field_metadata.get("PARQUET:doc", field_metadata.get("doc"))
+
+    if field_id_str := field_metadata.get("PARQUET:field_id", field_metadata.get("field_id")):
+        field_id = int(field_id_str)
+
+    return field_id, doc
+
+
+class _ConvertToIceberg(PyArrowSchemaVisitor[IcebergType], ABC):
+    def schema(self, schema: pa.Schema, field_results: List[Optional[IcebergType]]) -> Schema:
+        fields = []
+        for i, _ in enumerate(schema.names):
+            field = schema.field(i)
+            field_id, field_doc = _get_field_id_and_doc(field)
+            field_type = field_results[i]
+            if field_type is not None and field_id is not None:
+                fields.append(NestedField(field_id, field.name, field_type, required=not field.nullable, doc=field_doc))
+        return Schema(*fields)
+
+    def struct(self, struct: pa.StructType, field_results: List[Optional[IcebergType]]) -> IcebergType:
+        fields = []
+        for i in range(struct.num_fields):

Review Comment:
   Checked locally, and the struct itself is iterable:
   ```suggestion
           for i, field in enumerate(struct):
   ```



##########
python/tests/io/test_pyarrow.py:
##########
@@ -18,73 +18,19 @@
 
 import os
 import tempfile
-from typing import Any, List, Optional

Review Comment:
   Oeeeh, thanks for splitting this. It was getting too big indeed. A lot of rebasing for me ahead



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -484,6 +493,204 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results: List[Optional[IcebergType]] = []
+    for i, _ in enumerate(schema.names):

Review Comment:
   ```suggestion
       for i, field in enumerate(schema.fields):
   ```



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

To unsubscribe, e-mail: 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] JonasJ-ap commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6997:
URL: https://github.com/apache/iceberg/pull/6997#discussion_r1133326398


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -484,6 +493,204 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results: List[Optional[IcebergType]] = []
+    for i, _ in enumerate(schema.names):
+        field = schema.field(i)
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(Schema | pa.DataType): An instance of a Schema or an IcebergType
+        visitor (PyArrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    struct_results: List[Optional[T]] = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyArrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""
+
+    def after_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a list element."""
+
+    def before_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map key."""
+
+    def after_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map key."""
+
+    def before_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map value."""
+
+    def after_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map value."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[Optional[T]]) -> Schema:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[Optional[T]]) -> T:
+        """visit a struct"""
+
+    @abstractmethod
+    def list(self, list_type: pa.ListType, element_result: Optional[T]) -> Optional[T]:
+        """visit a list"""
+
+    @abstractmethod
+    def map(self, map_type: pa.MapType, key_result: Optional[T], value_result: Optional[T]) -> Optional[T]:
+        """visit a map"""
+
+    @abstractmethod
+    def primitive(self, primitive: pa.DataType) -> T:
+        """visit a primitive type"""
+
+
+def _get_field_id_and_doc(field: pa.Field) -> Tuple[Optional[int], Optional[str]]:
+    field_metadata = {k.decode(): v.decode() for k, v in field.metadata.items()}
+    field_id = None
+    doc = field_metadata.get("PARQUET:doc", field_metadata.get("doc"))
+
+    if field_id_str := field_metadata.get("PARQUET:field_id", field_metadata.get("field_id")):
+        field_id = int(field_id_str)
+
+    return field_id, doc
+
+
+class _ConvertToIceberg(PyArrowSchemaVisitor[IcebergType], ABC):
+    def schema(self, schema: pa.Schema, field_results: List[Optional[IcebergType]]) -> Schema:
+        fields = []
+        for i, _ in enumerate(schema.names):

Review Comment:
   Thank you for your suggestion and tests. I did not notice that pa.Schema is also iterable. I've updated the code and also verified that it worked locally.



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -484,6 +493,204 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results: List[Optional[IcebergType]] = []
+    for i, _ in enumerate(schema.names):
+        field = schema.field(i)
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(Schema | pa.DataType): An instance of a Schema or an IcebergType
+        visitor (PyArrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    struct_results: List[Optional[T]] = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyArrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""
+
+    def after_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a list element."""
+
+    def before_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map key."""
+
+    def after_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map key."""
+
+    def before_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map value."""
+
+    def after_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map value."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[Optional[T]]) -> Schema:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[Optional[T]]) -> T:
+        """visit a struct"""
+
+    @abstractmethod
+    def list(self, list_type: pa.ListType, element_result: Optional[T]) -> Optional[T]:
+        """visit a list"""
+
+    @abstractmethod
+    def map(self, map_type: pa.MapType, key_result: Optional[T], value_result: Optional[T]) -> Optional[T]:
+        """visit a map"""
+
+    @abstractmethod
+    def primitive(self, primitive: pa.DataType) -> T:
+        """visit a primitive type"""
+
+
+def _get_field_id_and_doc(field: pa.Field) -> Tuple[Optional[int], Optional[str]]:
+    field_metadata = {k.decode(): v.decode() for k, v in field.metadata.items()}
+    field_id = None
+    doc = field_metadata.get("PARQUET:doc", field_metadata.get("doc"))
+
+    if field_id_str := field_metadata.get("PARQUET:field_id", field_metadata.get("field_id")):
+        field_id = int(field_id_str)
+
+    return field_id, doc
+
+
+class _ConvertToIceberg(PyArrowSchemaVisitor[IcebergType], ABC):
+    def schema(self, schema: pa.Schema, field_results: List[Optional[IcebergType]]) -> Schema:
+        fields = []
+        for i, _ in enumerate(schema.names):

Review Comment:
   Thank you for your suggestion and tests. I did not notice that `pa.Schema` is also iterable. I've updated the code and also verified that it worked locally.



-- 
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 #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -476,6 +483,202 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results = []
+    for i in range(len(schema.names)):
+        field = schema.field(i)
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(Schema | IcebergType): An instance of a Schema or an IcebergType
+        visitor (PyarrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    struct_results = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyarrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""
+
+    def after_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a list element."""
+
+    def before_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map key."""
+
+    def after_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map key."""
+
+    def before_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map value."""
+
+    def after_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map value."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[T]) -> Schema:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[T]) -> T:
+        """visit a struct"""
+
+    @abstractmethod
+    def list(self, list_type: pa.ListType, element_result: T) -> T:
+        """visit a list"""
+
+    @abstractmethod
+    def map(self, map_type: pa.MapType, key_result: T, value_result: T) -> T:
+        """visit a map"""
+
+    @abstractmethod
+    def primitive(self, primitive: pa.DataType) -> T:
+        """visit a primitive type"""
+
+
+def _get_field_id(field: pa.Field) -> int:
+    if field.metadata is not None:
+        field_metadata = {k.decode(): v.decode() for k, v in field.metadata.items()}
+        if field_id := field_metadata.get("PARQUET:field_id"):
+            return int(field_id)
+    raise ValueError(f"Field {field.name} does not have a field_id")
+
+
+class _ConvertToIceberg(PyarrowSchemaVisitor[IcebergType], ABC):
+    def schema(self, schema: pa.Schema, field_results: List[IcebergType]) -> Schema:
+        fields = []
+        for i in range(len(schema.names)):
+            field = schema.field(i)
+            field_id = _get_field_id(field)
+            field_type = field_results[i]
+            if field_type is not None:
+                fields.append(NestedField(field_id, field.name, field_type, required=not field.nullable))
+        return Schema(*fields)
+
+    def struct(self, struct: pa.StructType, field_results: List[IcebergType]) -> IcebergType:
+        fields = []
+        for i in range(struct.num_fields):
+            field = struct[i]
+            field_id = _get_field_id(field)
+            # may need to check doc strings
+            field_type = field_results[i]
+            if field_type is not None:
+                fields.append(NestedField(field_id, field.name, field_type, required=not field.nullable))
+        return StructType(*fields)
+
+    def list(self, list_type: pa.ListType, element_result: IcebergType) -> IcebergType:
+        element_field = list_type.value_field
+        element_id = _get_field_id(element_field)
+        if element_result is not None:
+            return ListType(element_id, element_result, element_required=not element_field.nullable)
+        raise ValueError(f"List type must have element field: {list_type}")
+
+    def map(self, map_type: pa.MapType, key_result: IcebergType, value_result: IcebergType) -> IcebergType:
+        key_field = map_type.key_field
+        key_id = _get_field_id(key_field)
+        value_field = map_type.item_field
+        value_id = _get_field_id(value_field)
+        if key_result is not None and value_result is not None:
+            return MapType(key_id, key_result, value_id, value_result, value_required=not value_field.nullable)
+        raise ValueError(f"Map type must have key and value fields: {map_type}")
+
+    def primitive(self, primitive: pa.DataType) -> IcebergType:
+        if pa.types.is_boolean(primitive):
+            return BooleanType()
+        elif pa.types.is_int32(primitive) or pa.types.is_uint32(primitive):
+            return IntegerType()
+        elif pa.types.is_int64(primitive) or pa.types.is_uint64(primitive):
+            return LongType()
+        elif pa.types.is_float32(primitive):
+            return FloatType()
+        elif pa.types.is_float64(primitive):
+            return DoubleType()
+        elif pa.types.is_decimal(primitive):
+            if isinstance(primitive, pa.Decimal256Type):
+                primitive = cast(pa.Decimal256Type, primitive)
+            else:
+                primitive = cast(pa.Decimal128Type, primitive)
+            return DecimalType(primitive.precision, primitive.scale)
+        elif pa.types.is_string(primitive):
+            return StringType()
+        elif pa.types.is_date(primitive):
+            return DateType()
+        elif pa.types.is_time(primitive):
+            return TimeType()
+        elif pa.types.is_timestamp(primitive):
+            primitive = cast(pa.TimestampType, primitive)
+            if primitive.tz is not None:

Review Comment:
   We can handle non-utc timezones



-- 
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] JonasJ-ap commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6997:
URL: https://github.com/apache/iceberg/pull/6997#discussion_r1133368110


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -356,14 +366,19 @@ def field(self, field: NestedField, field_result: pa.DataType) -> pa.Field:
             name=field.name,
             type=field_result,
             nullable=field.optional,
-            metadata={"doc": field.doc, "id": str(field.field_id)} if field.doc else {},
+            metadata={PYTHON_DOC.decode(): field.doc, PYTHON_FIELD_ID.decode(): str(field.field_id)}
+            if field.doc
+            else {PYTHON_FIELD_ID.decode(): str(field.field_id)},

Review Comment:
   Thanks for the suggestion. I changed to the example's implementation



-- 
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] JonasJ-ap commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6997:
URL: https://github.com/apache/iceberg/pull/6997#discussion_r1146934576


##########
python/tests/test_integration.py:
##########
@@ -118,3 +123,22 @@ def test_ray_all_types(table_test_all_types: Table) -> None:
     pandas_dataframe = table_test_all_types.scan().to_pandas()
     assert ray_dataset.count() == pandas_dataframe.shape[0]
     assert pandas_dataframe.equals(ray_dataset.to_pandas())
+
+
+@pytest.mark.integration
+def test_pyarrow_to_iceberg_all_types(table_test_all_types: Table) -> None:
+    fs = S3FileSystem(
+        **{
+            "endpoint_override": "http://localhost:9000",
+            "access_key": "admin",
+            "secret_key": "password",
+        }
+    )
+    data_file_paths = [task.file.file_path for task in table_test_all_types.scan().plan_files()]
+    for data_file_path in data_file_paths:
+        uri = urlparse(data_file_path)
+        with fs.open_input_file(f"{uri.netloc}{uri.path}") as fout:
+            parquet_schema = pq.read_schema(fout)
+            stored_iceberg_schema = Schema.parse_raw(parquet_schema.metadata.get(b"iceberg.schema"))
+            converted_iceberg_schema = pyarrow_to_schema(parquet_schema)
+            assert converted_iceberg_schema == stored_iceberg_schema

Review Comment:
   Rebase to the master branch. Since python integration tests are in, we can now test pyarrow to iceberg schema visitor on real parquet 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 #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -476,6 +483,202 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results = []
+    for i in range(len(schema.names)):
+        field = schema.field(i)
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(Schema | IcebergType): An instance of a Schema or an IcebergType
+        visitor (PyarrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    struct_results = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyarrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""
+
+    def after_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a list element."""
+
+    def before_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map key."""
+
+    def after_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map key."""
+
+    def before_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map value."""
+
+    def after_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map value."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[T]) -> Schema:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[T]) -> T:
+        """visit a struct"""
+
+    @abstractmethod
+    def list(self, list_type: pa.ListType, element_result: T) -> T:
+        """visit a list"""
+
+    @abstractmethod
+    def map(self, map_type: pa.MapType, key_result: T, value_result: T) -> T:
+        """visit a map"""
+
+    @abstractmethod
+    def primitive(self, primitive: pa.DataType) -> T:
+        """visit a primitive type"""
+
+
+def _get_field_id(field: pa.Field) -> int:
+    if field.metadata is not None:
+        field_metadata = {k.decode(): v.decode() for k, v in field.metadata.items()}
+        if field_id := field_metadata.get("PARQUET:field_id"):
+            return int(field_id)
+    raise ValueError(f"Field {field.name} does not have a field_id")
+
+
+class _ConvertToIceberg(PyarrowSchemaVisitor[IcebergType], ABC):
+    def schema(self, schema: pa.Schema, field_results: List[IcebergType]) -> Schema:
+        fields = []
+        for i in range(len(schema.names)):
+            field = schema.field(i)
+            field_id = _get_field_id(field)
+            field_type = field_results[i]
+            if field_type is not None:
+                fields.append(NestedField(field_id, field.name, field_type, required=not field.nullable))
+        return Schema(*fields)
+
+    def struct(self, struct: pa.StructType, field_results: List[IcebergType]) -> IcebergType:
+        fields = []
+        for i in range(struct.num_fields):
+            field = struct[i]
+            field_id = _get_field_id(field)
+            # may need to check doc strings
+            field_type = field_results[i]
+            if field_type is not None:
+                fields.append(NestedField(field_id, field.name, field_type, required=not field.nullable))
+        return StructType(*fields)
+
+    def list(self, list_type: pa.ListType, element_result: IcebergType) -> IcebergType:
+        element_field = list_type.value_field
+        element_id = _get_field_id(element_field)
+        if element_result is not None:
+            return ListType(element_id, element_result, element_required=not element_field.nullable)
+        raise ValueError(f"List type must have element field: {list_type}")
+
+    def map(self, map_type: pa.MapType, key_result: IcebergType, value_result: IcebergType) -> IcebergType:
+        key_field = map_type.key_field
+        key_id = _get_field_id(key_field)
+        value_field = map_type.item_field
+        value_id = _get_field_id(value_field)
+        if key_result is not None and value_result is not None:
+            return MapType(key_id, key_result, value_id, value_result, value_required=not value_field.nullable)
+        raise ValueError(f"Map type must have key and value fields: {map_type}")
+
+    def primitive(self, primitive: pa.DataType) -> IcebergType:
+        if pa.types.is_boolean(primitive):
+            return BooleanType()
+        elif pa.types.is_int32(primitive) or pa.types.is_uint32(primitive):
+            return IntegerType()
+        elif pa.types.is_int64(primitive) or pa.types.is_uint64(primitive):
+            return LongType()
+        elif pa.types.is_float32(primitive):
+            return FloatType()
+        elif pa.types.is_float64(primitive):
+            return DoubleType()
+        elif pa.types.is_decimal(primitive):
+            if isinstance(primitive, pa.Decimal256Type):
+                primitive = cast(pa.Decimal256Type, primitive)
+            else:
+                primitive = cast(pa.Decimal128Type, primitive)
+            return DecimalType(primitive.precision, primitive.scale)
+        elif pa.types.is_string(primitive):
+            return StringType()
+        elif pa.types.is_date(primitive):
+            return DateType()
+        elif pa.types.is_time(primitive):

Review Comment:
   This still seems suspicious to me. While it's now correct, how does PyArrow read Python files that have other time or timestamp 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] rdblue commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -486,6 +499,207 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results: List[Optional[IcebergType]] = []
+    for field in schema:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)

Review Comment:
   Why isn't this done in the `visit_pyarrow` method? It seems incorrect to have top-level logic that must be copied into every function that uses a visitor.
   
   I'd expect something like this:
   
   ```python
   @visit_pyarrow.register(pa.Schema)
   def _(schema: pa.Schema, visitor: PyArrowSchemaVisitor[T]) -> T
       field_results: List[Optional[T]] = []
       for field in schema:
           ...
       return visitor.schema(schema, field_results)
   ```



-- 
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 #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -358,14 +368,17 @@ def field(self, field: NestedField, field_result: pa.DataType) -> pa.Field:
             name=field.name,
             type=field_result,
             nullable=field.optional,
-            metadata={"doc": field.doc, "id": str(field.field_id)} if field.doc else {},
+            metadata={FIELD_DOC: field.doc, FIELD_ID: str(field.field_id)} if field.doc else {FIELD_ID: str(field.field_id)},
         )
 
-    def list(self, _: ListType, element_result: pa.DataType) -> pa.DataType:
-        return pa.list_(value_type=element_result)
+    def list(self, list_type: ListType, element_result: pa.DataType) -> pa.DataType:
+        element_field = self.field(list_type.element_field, element_result)
+        return pa.list_(value_type=element_field)

Review Comment:
   `list_` appears to accept `DataType | Field` but it doesn't look like `map_` does the same. When I inspect the method it expects `DataType` for both `key_type` and `value_type`. Is there a test that this is doing what we expect?



-- 
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 #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -476,6 +483,217 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    return visit_arrow_schema(schema, _ConvertToIceberg())
+
+
+def visit_arrow_schema(obj: pa.Schema, visitor: ArrowSchemaVisitor[T]) -> Schema:
+    struct_results = []
+    for i in range(len(obj.names)):
+        field = obj.field(i)
+        visitor.before_field(field)
+        struct_result = visit_arrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.schema(obj, struct_results)
+
+
+def visit_arrow(obj: pa.DataType, visitor: ArrowSchemaVisitor[T]) -> T:
+    if pa.types.is_struct(obj):
+        return visit_arrow_struct(obj, visitor)
+    elif pa.types.is_list(obj):
+        return visit_arrow_list(obj, visitor)
+    elif pa.types.is_map(obj):
+        return visit_arrow_map(obj, visitor)
+    else:
+        return visit_arrow_primitive(obj, visitor)
+
+
+def visit_arrow_struct(obj: pa.DataType, visitor: ArrowSchemaVisitor[T]) -> T:
+    if not pa.types.is_struct(obj):
+        raise TypeError(f"Expected struct type, got {type(obj)}")
+    obj = cast(pa.StructType, obj)
+    struct_results = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_arrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+def visit_arrow_list(obj: pa.DataType, visitor: ArrowSchemaVisitor[T]) -> T:
+    if not pa.types.is_list(obj):
+        raise TypeError(f"Expected list type, got {type(obj)}")
+    obj = cast(pa.ListType, obj)
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_arrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+def visit_arrow_map(obj: pa.DataType, visitor: ArrowSchemaVisitor[T]) -> T:
+    if not pa.types.is_map(obj):
+        raise TypeError(f"Expected map type, got {type(obj)}")
+    obj = cast(pa.MapType, obj)
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_arrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_arrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+def visit_arrow_primitive(obj: pa.DataType, visitor: ArrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class ArrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""
+
+    def after_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a list element."""
+
+    def before_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map key."""
+
+    def after_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map key."""
+
+    def before_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map value."""
+
+    def after_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map value."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[T]) -> Schema:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[T]) -> T:
+        """visit a struct"""
+
+    @abstractmethod
+    def list(self, list_type: pa.ListType, element_result: T) -> T:
+        """visit a list"""
+
+    @abstractmethod
+    def map(self, map_type: pa.MapType, key_result: T, value_result: T) -> T:
+        """visit a map"""
+
+    @abstractmethod
+    def primitive(self, primitive: pa.DataType) -> T:
+        """visit a primitive type"""
+
+
+def _get_field_id(field: pa.Field) -> int:
+    field_metadata = {k.decode(): v.decode() for k, v in field.metadata.items()}
+    if field_id := field_metadata.get("PARQUET:field_id"):
+        return field_id
+    raise ValueError(f"Field {field.name} does not have a field_id")
+
+
+class _ConvertToIceberg(ArrowSchemaVisitor[IcebergType], ABC):
+    def schema(self, schema: pa.Schema, field_results: List[IcebergType]) -> Schema:
+        fields = []
+        for i in range(len(schema.names)):
+            field = schema.field(i)
+            field_id = _get_field_id(field)
+            field_type = field_results[i]
+            if field_id is not None and field_type is not None:
+                if field.nullable:
+                    fields.append(NestedField(field_id, field.name, field_type, False))
+                else:
+                    fields.append(NestedField(field_id, field.name, field_type, True))
+        return Schema(*fields)
+
+    def struct(self, struct: pa.StructType, field_results: List[IcebergType]) -> IcebergType:
+        fields = []
+        for i in range(struct.num_fields):
+            field = struct[i]
+            field_id = _get_field_id(field)
+            # may need to check doc strings
+            field_type = field_results[i]
+            if field_id is not None and field_type is not None:
+                if field.nullable:
+                    fields.append(NestedField(field_id, field.name, field_type, False))
+                else:
+                    fields.append(NestedField(field_id, field.name, field_type, True))
+        return StructType(*fields)
+
+    def list(self, list_type: pa.ListType, element_result: IcebergType) -> IcebergType:
+        element_field = list_type.value_field
+        element_id = _get_field_id(element_field)
+        if element_id is not None and element_result is not None:
+            if element_field.nullable:
+                return ListType(element_id, element_result, False)
+            else:
+                return ListType(element_id, element_result, True)
+        raise ValueError("List type must have element field")

Review Comment:
   What do you think of including the input type in the exception to help the user:
   ```suggestion
           raise ValueError(f"List type must have element field: {list_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] Fokko commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -492,11 +710,7 @@ def _file_to_table(
         schema_raw = None
         if metadata := parquet_schema.metadata:
             schema_raw = metadata.get(ICEBERG_SCHEMA)
-        if schema_raw is None:
-            raise ValueError(
-                "Iceberg schema is not embedded into the Parquet file, see https://github.com/apache/iceberg/issues/6505"
-            )
-        file_schema = Schema.parse_raw(schema_raw)
+        file_schema = Schema.parse_raw(schema_raw) if schema_raw is not None else pyarrow_to_schema(parquet_schema)

Review Comment:
   If the `field-id`'s are not set, I believe there is another fallback level where we look at the names and try to look them up in the table schema. That should be done in a separate PR, but maybe good to add a comment 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] Fokko commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -15,6 +15,7 @@
 # specific language governing permissions and limitations
 # under the License.
 # pylint: disable=redefined-outer-name,arguments-renamed

Review Comment:
   Instead of adding another line, you can just append the `W0511` to this
   
   ```suggestion
   # pylint: disable=redefined-outer-name,arguments-renamed,fixme
   ```



##########
python/tests/io/test_pyarrow.py:
##########
@@ -1130,3 +1131,15 @@ def test_projection_filter_on_unknown_field(schema_int_str: Schema, file_int_str
         _ = project(schema, [file_int_str], GreaterThan("unknown_field", "1"), schema_int_str)
 
     assert "Could not find field with name unknown_field, case_sensitive=True" in str(exc_info.value)
+
+
+def test_pyarrow_to_schema_simple(table_schema_simple: Schema, pyarrow_schema_simple: pa.Schema) -> None:
+    actual = str(pyarrow_to_schema(pyarrow_schema_simple))
+    expected = str(table_schema_simple)
+    assert actual == expected
+
+
+def test_pyarrow_to_schema_nested(table_schema_nested: Schema, pyarrow_schema_nested: pa.Schema) -> None:
+    actual = str(pyarrow_to_schema(pyarrow_schema_nested))

Review Comment:
   Thank you for giving this a try. I did some testing on my end, and adding the fields works perfectly fine for Arrow. I've created a table, and read that using PyIceberg:
   
   ```sql
   CREATE TABLE default.maps_and_lists(
       some_list ARRAY<string>,
       some_amp  MAP<string, int>
   )
   
   INSERT INTO default.maps_and_lists VALUES(array("a", "b", "c"), map("a", 1, "b", 2, "c", 3))
   ```
   Wit the changes that you suggested, I can just read it into a dataframe:
   
   ```python
   def test_vo():
       from pyiceberg.catalog import load_catalog
       cat = load_catalog('local')
       tbl = cat.load_table('default.maps_and_lists')
       df = tbl.scan().to_arrow()
       print(df)
   ```
   
   Which shows:
   
   ```
   some_list: list<item: string>
     child 0, item: string
   some_amp: map<string, int32>
     child 0, entries: struct<key: string not null, value: int32> not null
         child 0, key: string not null
         child 1, value: int32
   ----
   some_list: [[["a","b","c"]]]
   some_amp: [[keys:["a","b","c"]values:[1,2,3]]]
   ```
   I'm in favor of changing this.



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -476,6 +483,202 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results = []
+    for i in range(len(schema.names)):
+        field = schema.field(i)
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(Schema | IcebergType): An instance of a Schema or an IcebergType
+        visitor (PyarrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    struct_results = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyarrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""
+
+    def after_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a list element."""
+
+    def before_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map key."""
+
+    def after_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map key."""
+
+    def before_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map value."""
+
+    def after_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map value."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[T]) -> Schema:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[T]) -> T:
+        """visit a struct"""
+
+    @abstractmethod
+    def list(self, list_type: pa.ListType, element_result: T) -> T:
+        """visit a list"""
+
+    @abstractmethod
+    def map(self, map_type: pa.MapType, key_result: T, value_result: T) -> T:
+        """visit a map"""
+
+    @abstractmethod
+    def primitive(self, primitive: pa.DataType) -> T:
+        """visit a primitive type"""
+
+
+def _get_field_id(field: pa.Field) -> int:
+    if field.metadata is not None:
+        field_metadata = {k.decode(): v.decode() for k, v in field.metadata.items()}
+        if field_id := field_metadata.get("PARQUET:field_id"):

Review Comment:
   There will be other keys for ORC and Avro. I think we can start with this.



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -476,6 +483,202 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results = []
+    for i in range(len(schema.names)):
+        field = schema.field(i)
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(Schema | IcebergType): An instance of a Schema or an IcebergType

Review Comment:
   ```suggestion
           obj(Schema | pa.DataType): An instance of a Schema or an IcebergType
   ```



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -492,11 +710,7 @@ def _file_to_table(
         schema_raw = None
         if metadata := parquet_schema.metadata:
             schema_raw = metadata.get(ICEBERG_SCHEMA)
-        if schema_raw is None:
-            raise ValueError(
-                "Iceberg schema is not embedded into the Parquet file, see https://github.com/apache/iceberg/issues/6505"
-            )
-        file_schema = Schema.parse_raw(schema_raw)
+        file_schema = Schema.parse_raw(schema_raw) if schema_raw is not None else pyarrow_to_schema(parquet_schema)

Review Comment:
   Exactly!



##########
python/tests/io/test_pyarrow.py:
##########
@@ -1130,3 +1131,15 @@ def test_projection_filter_on_unknown_field(schema_int_str: Schema, file_int_str
         _ = project(schema, [file_int_str], GreaterThan("unknown_field", "1"), schema_int_str)
 
     assert "Could not find field with name unknown_field, case_sensitive=True" in str(exc_info.value)
+
+
+def test_pyarrow_to_schema_simple(table_schema_simple: Schema, pyarrow_schema_simple: pa.Schema) -> None:
+    actual = str(pyarrow_to_schema(pyarrow_schema_simple))
+    expected = str(table_schema_simple)
+    assert actual == expected
+
+
+def test_pyarrow_to_schema_nested(table_schema_nested: Schema, pyarrow_schema_nested: pa.Schema) -> None:
+    actual = str(pyarrow_to_schema(pyarrow_schema_nested))
+    expected = str(table_schema_nested)

Review Comment:
   Personally, I prefer to just turn the expected in the actual string:
   ```python
   def test_pyarrow_to_schema_nested(table_schema_nested: Schema, pyarrow_schema_nested: pa.Schema) -> None:
       actual = str(pyarrow_to_schema(pyarrow_schema_nested))
       assert actual == """table {
     1: foo: optional string
     2: bar: required int
     3: baz: optional boolean
     4: qux: required list<string>
     6: quux: required map<string, map<string, int>>
     11: location: required list<struct<13: latitude: optional float, 14: longitude: optional float>>
     15: person: optional struct<16: name: optional string, 17: age: required int>
   }"""
   
   This way you can see what you're asserting. Gives me a sense of confidence :)
   ```



##########
python/tests/io/test_pyarrow.py:
##########
@@ -1130,3 +1131,15 @@ def test_projection_filter_on_unknown_field(schema_int_str: Schema, file_int_str
         _ = project(schema, [file_int_str], GreaterThan("unknown_field", "1"), schema_int_str)
 
     assert "Could not find field with name unknown_field, case_sensitive=True" in str(exc_info.value)
+
+
+def test_pyarrow_to_schema_simple(table_schema_simple: Schema, pyarrow_schema_simple: pa.Schema) -> None:
+    actual = str(pyarrow_to_schema(pyarrow_schema_simple))
+    expected = str(table_schema_simple)
+    assert actual == expected
+
+
+def test_pyarrow_to_schema_nested(table_schema_nested: Schema, pyarrow_schema_nested: pa.Schema) -> None:
+    actual = str(pyarrow_to_schema(pyarrow_schema_nested))
+    expected = str(table_schema_nested)

Review Comment:
   I don't think the schema-id is encoded in the Parquet file, but that's okay. The validation was done on write time, and on read time, we turn the schema into the requested schema. Not having the `identifier_field_ids` is okay, it should be provided by the table schema.



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -476,6 +483,202 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results = []
+    for i in range(len(schema.names)):
+        field = schema.field(i)
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(Schema | IcebergType): An instance of a Schema or an IcebergType
+        visitor (PyarrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    struct_results = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyarrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""
+
+    def after_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a list element."""
+
+    def before_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map key."""
+
+    def after_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map key."""
+
+    def before_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map value."""
+
+    def after_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map value."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[T]) -> Schema:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[T]) -> T:
+        """visit a struct"""
+
+    @abstractmethod
+    def list(self, list_type: pa.ListType, element_result: T) -> T:
+        """visit a list"""
+
+    @abstractmethod
+    def map(self, map_type: pa.MapType, key_result: T, value_result: T) -> T:
+        """visit a map"""
+
+    @abstractmethod
+    def primitive(self, primitive: pa.DataType) -> T:
+        """visit a primitive type"""
+
+
+def _get_field_id(field: pa.Field) -> int:
+    if field.metadata is not None:
+        field_metadata = {k.decode(): v.decode() for k, v in field.metadata.items()}
+        if field_id := field_metadata.get("PARQUET:field_id"):
+            return int(field_id)
+    raise ValueError(f"Field {field.name} does not have a field_id")

Review Comment:
   Nit, we use the following pattern quite a lot in PyIceberg:
   ```suggestion
       raise ValueError(f"Field does not have a field_id: {field.name}")
   ```



##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -476,6 +483,202 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results = []
+    for i in range(len(schema.names)):
+        field = schema.field(i)
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(Schema | IcebergType): An instance of a Schema or an IcebergType
+        visitor (PyarrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    struct_results = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyarrowSchemaVisitor(Generic[T], ABC):

Review Comment:
   This is the casing that PyArrow also uses.
   ```suggestion
   class PyArrowSchemaVisitor(Generic[T], ABC):
   ```



-- 
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 #6997: Python: Infer Iceberg schema from the Parquet file

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -476,6 +483,202 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results = []
+    for i in range(len(schema.names)):
+        field = schema.field(i)
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(Schema | IcebergType): An instance of a Schema or an IcebergType
+        visitor (PyarrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    struct_results = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyarrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyarrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""
+
+    def after_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a list element."""
+
+    def before_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map key."""
+
+    def after_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map key."""
+
+    def before_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map value."""
+
+    def after_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map value."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[T]) -> Schema:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[T]) -> T:
+        """visit a struct"""
+
+    @abstractmethod
+    def list(self, list_type: pa.ListType, element_result: T) -> T:
+        """visit a list"""
+
+    @abstractmethod
+    def map(self, map_type: pa.MapType, key_result: T, value_result: T) -> T:
+        """visit a map"""
+
+    @abstractmethod
+    def primitive(self, primitive: pa.DataType) -> T:
+        """visit a primitive type"""
+
+
+def _get_field_id(field: pa.Field) -> int:
+    if field.metadata is not None:
+        field_metadata = {k.decode(): v.decode() for k, v in field.metadata.items()}
+        if field_id := field_metadata.get("PARQUET:field_id"):
+            return int(field_id)
+    raise ValueError(f"Field {field.name} does not have a field_id")
+
+
+class _ConvertToIceberg(PyarrowSchemaVisitor[IcebergType], ABC):
+    def schema(self, schema: pa.Schema, field_results: List[IcebergType]) -> Schema:
+        fields = []
+        for i in range(len(schema.names)):

Review Comment:
   Can you use `enumerate(schema.fields)` instead of `range`?



-- 
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] JonasJ-ap commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6997:
URL: https://github.com/apache/iceberg/pull/6997#discussion_r1174068183


##########
python/tests/io/test_pyarrow.py:
##########
@@ -18,73 +18,19 @@
 
 import os
 import tempfile
-from typing import Any, List, Optional

Review Comment:
   Sure. I reverted the refactoring to `test_pyarrow`. Will open a new PR soon



-- 
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] JonasJ-ap commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6997:
URL: https://github.com/apache/iceberg/pull/6997#discussion_r1174109107


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -486,6 +499,207 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    struct_results: List[Optional[IcebergType]] = []
+    for field in schema:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+    return visitor.schema(schema, struct_results)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(pa.DataType): An instance of a Schema or an IcebergType
+        visitor (PyArrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    struct_results: List[Optional[T]] = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_list_element(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_list_element(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_map_key(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_map_key(obj.key_field)
+    visitor.before_map_value(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_map_value(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> T:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got: {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyArrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    def before_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a list element."""
+
+    def after_list_element(self, element: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a list element."""
+
+    def before_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map key."""
+
+    def after_map_key(self, key: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map key."""
+
+    def before_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a map value."""
+
+    def after_map_value(self, value: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a map value."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[Optional[T]]) -> Schema:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[Optional[T]]) -> T:

Review Comment:
   Thanks for pointing this out. For these abstract methods, I think they shall return `Optional[T]`



-- 
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] JonasJ-ap commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6997:
URL: https://github.com/apache/iceberg/pull/6997#discussion_r1174167581


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -122,6 +126,12 @@
 ONE_MEGABYTE = 1024 * 1024
 BUFFER_SIZE = "buffer-size"
 ICEBERG_SCHEMA = b"iceberg.schema"
+FIELD_ID = "field_id"
+FIELD_DOC = "doc"
+PYARROW_FIELD_ID_KEYS = [b"PARQUET:field_id", b"field_id"]

Review Comment:
   I added support for the non-prefixed `field_id` to maintain consistency with the `schema_to_pyarrow` visitor, which uses `field_id` as the metadata key. This support may be helpful in the future if we use this visitor to get the `pa.Schema` for writing data files. Additionally, it allows us to perform round-trip tests by combining `schema_to_pyarrow` and `pyarrow_to_schema`.
   
   The `pyarrow_to_schema` visitor first looks for `PARQUET:field_id` and then falls back to `field_id`, so the behavior is as expected. However, if you still think the non-prefixed `field_id` should not be supported, please let me know.



-- 
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] JonasJ-ap commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6997:
URL: https://github.com/apache/iceberg/pull/6997#discussion_r1131163621


##########
python/tests/conftest.py:
##########
@@ -116,6 +117,17 @@ def table_schema_simple() -> Schema:
     )
 
 
+@pytest.fixture(scope="session")
+def pyarrow_schema_simple() -> pa.Schema:

Review Comment:
   I applied the change discussed in https://github.com/apache/iceberg/pull/6997#discussion_r1125317095, so now I am able to do a round-trip unit test here. I also added several tests for individual types
   
   For real parquet file test, I downloaded one from my test table on AWS and perform the following test:
   ```python
   parquet_test_path = "/Users/jonasjiang/.CMVolumes/gluetestjonas/warehouse/iceberg_ref.db/nested_frame_unpartitioned_parquet/data/00002-17-e809caf4-2060-4319-b299-a2caf0dd133d-00001.parquet"
   fs = LocalFileSystem()
   def test_pyarrow_to_iceberg(path):
       with fs.open_input_file(path) as f:
           parquet_schema = pq.read_schema(f)
           converted_iceberg_schema = pyarrow_to_schema(parquet_schema)
           stored_iceberg_schema = Schema.parse_raw(parquet_schema.metadata[b'iceberg.schema'].decode())
           print(stored_iceberg_schema)
           assert converted_iceberg_schema == stored_iceberg_schema
           print("Verified")
   test_pyarrow_to_iceberg(parquet_test_path)
   ```
   The output:
   ```python
   table {
     1: id: required long
     2: longCol: required long
     3: decimalCol: optional decimal(10, 2)
     4: magic_number: required double
     5: dateCol: required date
     6: dateString: required string
     7: random1: optional long
     8: random2: optional long
     9: random3: optional long
     10: random4: optional long
     11: random5: optional long
     12: innerStruct1: required struct<23: random1: optional long, 24: random2: optional long>
     13: innerStruct2: required struct<25: random3: optional long, 26: random4: optional long>
     14: structCol1: required struct<27: innerStruct1: required struct<29: random1: optional long, 30: random2: optional long>, 28: innerStruct2: required struct<31: random3: optional long, 32: random4: optional long>>
     15: innerStruct3: required struct<33: col1: optional string, 34: col2: optional string>
     16: structCol2: required struct<35: innerStruct3: required struct<37: col1: optional string, 38: col2: optional string>, 36: col2: required struct<39: col1: optional string, 40: col2: optional string>>
     17: arrayCol: required list<long>
     18: arrayStructCol: required list<struct<43: random1: optional long, 44: random2: optional long>>
     19: mapCol1: required map<struct<47: innerStruct1: required struct<49: random1: optional long, 50: random2: optional long>, 48: innerStruct2: required struct<51: random3: optional long, 52: random4: optional long>>, struct<53: innerStruct3: required struct<55: col1: optional string, 56: col2: optional string>, 54: col2: required struct<57: col1: optional string, 58: col2: optional string>>>
     20: mapCol2: required map<long, string>
     21: mapCol3: required map<date, list<long>>
     22: structCol3: required struct<64: structCol2: required struct<67: innerStruct3: required struct<69: col1: optional string, 70: col2: optional string>, 68: col2: required struct<71: col1: optional string, 72: col2: optional string>>, 65: mapCol3: required map<date, list<long>>, 66: arrayCol: required list<long>>
   }
   Verified
   ```
   Please let me know if you want to see more related 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] sheinbergon commented on pull request #6997: Python: Infer Iceberg schema from the Parquet file

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

   @rdblue thank you for you response. Any ETA on the 0.4.0 releaase, BTW? (if there's a better place to ask, feel free to point me 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] JonasJ-ap commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6997:
URL: https://github.com/apache/iceberg/pull/6997#discussion_r1146934576


##########
python/tests/test_integration.py:
##########
@@ -118,3 +123,22 @@ def test_ray_all_types(table_test_all_types: Table) -> None:
     pandas_dataframe = table_test_all_types.scan().to_pandas()
     assert ray_dataset.count() == pandas_dataframe.shape[0]
     assert pandas_dataframe.equals(ray_dataset.to_pandas())
+
+
+@pytest.mark.integration
+def test_pyarrow_to_iceberg_all_types(table_test_all_types: Table) -> None:
+    fs = S3FileSystem(
+        **{
+            "endpoint_override": "http://localhost:9000",
+            "access_key": "admin",
+            "secret_key": "password",
+        }
+    )
+    data_file_paths = [task.file.file_path for task in table_test_all_types.scan().plan_files()]
+    for data_file_path in data_file_paths:
+        uri = urlparse(data_file_path)
+        with fs.open_input_file(f"{uri.netloc}{uri.path}") as fout:
+            parquet_schema = pq.read_schema(fout)
+            stored_iceberg_schema = Schema.parse_raw(parquet_schema.metadata.get(b"iceberg.schema"))
+            converted_iceberg_schema = pyarrow_to_schema(parquet_schema)
+            assert converted_iceberg_schema == stored_iceberg_schema

Review Comment:
   Rebase and add an integration test for the pyarrow to iceberg visitor.



-- 
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] JonasJ-ap commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6997:
URL: https://github.com/apache/iceberg/pull/6997#discussion_r1146934576


##########
python/tests/test_integration.py:
##########
@@ -118,3 +123,22 @@ def test_ray_all_types(table_test_all_types: Table) -> None:
     pandas_dataframe = table_test_all_types.scan().to_pandas()
     assert ray_dataset.count() == pandas_dataframe.shape[0]
     assert pandas_dataframe.equals(ray_dataset.to_pandas())
+
+
+@pytest.mark.integration
+def test_pyarrow_to_iceberg_all_types(table_test_all_types: Table) -> None:
+    fs = S3FileSystem(
+        **{
+            "endpoint_override": "http://localhost:9000",
+            "access_key": "admin",
+            "secret_key": "password",
+        }
+    )
+    data_file_paths = [task.file.file_path for task in table_test_all_types.scan().plan_files()]
+    for data_file_path in data_file_paths:
+        uri = urlparse(data_file_path)
+        with fs.open_input_file(f"{uri.netloc}{uri.path}") as fout:
+            parquet_schema = pq.read_schema(fout)
+            stored_iceberg_schema = Schema.parse_raw(parquet_schema.metadata.get(b"iceberg.schema"))
+            converted_iceberg_schema = pyarrow_to_schema(parquet_schema)
+            assert converted_iceberg_schema == stored_iceberg_schema

Review Comment:
   Rebase and add a integration test for the pyarrow to iceberg visitor.



-- 
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] JonasJ-ap commented on a diff in pull request #6997: Python: Infer Iceberg schema from the Parquet file

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6997:
URL: https://github.com/apache/iceberg/pull/6997#discussion_r1179568693


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -486,6 +499,195 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
     return boolean_expression_visit(expr, _ConvertToArrowExpression())
 
 
+def pyarrow_to_schema(schema: pa.Schema) -> Schema:
+    visitor = _ConvertToIceberg()
+    return visit_pyarrow(schema, visitor)
+
+
+@singledispatch
+def visit_pyarrow(obj: pa.DataType | pa.Schema, visitor: PyArrowSchemaVisitor[T]) -> T:
+    """A generic function for applying a pyarrow schema visitor to any point within a schema
+
+    The function traverses the schema in post-order fashion
+
+    Args:
+        obj(pa.DataType): An instance of a Schema or an IcebergType
+        visitor (PyArrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class
+
+    Raises:
+        NotImplementedError: If attempting to visit an unrecognized object type
+    """
+    raise NotImplementedError("Cannot visit non-type: %s" % obj)
+
+
+@visit_pyarrow.register(pa.Schema)
+def _(obj: pa.Schema, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    struct_results: List[Optional[T]] = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.schema(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.StructType)
+def _(obj: pa.StructType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    struct_results: List[Optional[T]] = []
+    for field in obj:
+        visitor.before_field(field)
+        struct_result = visit_pyarrow(field.type, visitor)
+        visitor.after_field(field)
+        struct_results.append(struct_result)
+
+    return visitor.struct(obj, struct_results)
+
+
+@visit_pyarrow.register(pa.ListType)
+def _(obj: pa.ListType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_field(obj.value_field)
+    list_result = visit_pyarrow(obj.value_field.type, visitor)
+    visitor.after_field(obj.value_field)
+    return visitor.list(obj, list_result)
+
+
+@visit_pyarrow.register(pa.MapType)
+def _(obj: pa.MapType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    visitor.before_field(obj.key_field)
+    key_result = visit_pyarrow(obj.key_field.type, visitor)
+    visitor.after_field(obj.key_field)
+    visitor.before_field(obj.item_field)
+    value_result = visit_pyarrow(obj.item_field.type, visitor)
+    visitor.after_field(obj.item_field)
+    return visitor.map(obj, key_result, value_result)
+
+
+@visit_pyarrow.register(pa.DataType)
+def _(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> Optional[T]:
+    if pa.types.is_nested(obj):
+        raise TypeError(f"Expected primitive type, got: {type(obj)}")
+    return visitor.primitive(obj)
+
+
+class PyArrowSchemaVisitor(Generic[T], ABC):
+    def before_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately before visiting a field."""
+
+    def after_field(self, field: pa.Field) -> None:
+        """Override this method to perform an action immediately after visiting a field."""
+
+    @abstractmethod
+    def schema(self, schema: pa.Schema, field_results: List[Optional[T]]) -> Optional[T]:
+        """visit a schema"""
+
+    @abstractmethod
+    def struct(self, struct: pa.StructType, field_results: List[Optional[T]]) -> Optional[T]:
+        """visit a struct"""
+
+    @abstractmethod
+    def list(self, list_type: pa.ListType, element_result: Optional[T]) -> Optional[T]:
+        """visit a list"""
+
+    @abstractmethod
+    def map(self, map_type: pa.MapType, key_result: Optional[T], value_result: Optional[T]) -> Optional[T]:
+        """visit a map"""
+
+    @abstractmethod
+    def primitive(self, primitive: pa.DataType) -> Optional[T]:
+        """visit a primitive type"""
+
+
+def _get_field_id(field: pa.Field) -> Optional[int]:
+    for pyarrow_field_id_key in PYARROW_FIELD_ID_KEYS:
+        if field_id_str := field.metadata.get(pyarrow_field_id_key):
+            return int(field_id_str.decode())
+    return None
+
+
+def _get_field_doc(field: pa.Field) -> Optional[str]:
+    for pyarrow_doc_key in PYARROW_FIELD_DOC_KEYS:
+        if doc_str := field.metadata.get(pyarrow_doc_key):
+            return doc_str.decode()
+    return None
+
+
+class _ConvertToIceberg(PyArrowSchemaVisitor[Union[IcebergType, Schema]]):
+    def schema(self, schema: pa.Schema, field_results: List[Optional[IcebergType]]) -> Schema:

Review Comment:
   Thank you very much for your suggestions and test. I've added it to the code



-- 
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