You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2022/05/10 14:20:11 UTC

[GitHub] [iceberg] Fokko opened a new pull request, #4742: Python: Convert Avro to Iceberg schema

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

   Converts an Avro schema into an Iceberg one.
   
   Looked at https://github.com/apache/iceberg/blob/master/python_legacy/iceberg/core/avro/avro_to_iceberg.py but I decided to rewrite it to make it more pythonic.


-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r872843543


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,274 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Util class for converting between Avro and Iceberg schemas
+
+"""
+import logging
+from typing import Any, Dict, List, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DoubleType,
+    FloatType,
+    IntegerType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+)
+
+logger = logging.getLogger(__name__)
+
+
+class AvroSchemaConversion:
+    PRIMITIVE_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "boolean": BooleanType(),
+        "bytes": BinaryType(),
+        "date": DateType(),
+        "double": DoubleType(),
+        "float": FloatType(),
+        "int": IntegerType(),
+        "long": LongType(),
+        "string": StringType(),
+        "time-millis": TimeType(),
+        "timestamp-millis": TimestampType(),
+    }
+
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expect to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into a Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+
+        Todo:
+            * Implement full support for unions
+            * Implement logical types
+        """
+        fields = self._parse_record(avro_schema)
+        return Schema(*fields.fields, schema_id=1)  # type: ignore
+
+    def _parse_record(self, avro_field: Dict[str, Any]) -> StructType:
+        return StructType(*[self._parse_field(field) for field in avro_field["fields"]])
+
+    def _resolve_union(self, type_union: Union[Dict, List, str]) -> Tuple[Union[str, dict], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        """
+        avro_types: Union[Dict, List]
+        if isinstance(type_union, str):
+            avro_types = [type_union]
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        # Filter the null value, so we know the actual type
+        avro_types = list(filter(lambda t: t != "null", avro_types))
+
+        if len(avro_types) != 1:
+            raise ValueError("Support for unions is yet to be implemented")
+
+        avro_type = avro_types[0]
+
+        return avro_type, is_optional
+
+    def _parse_field(self, field: Dict[str, Any]) -> NestedField:
+        """
+        Recursively walks through the Schema, constructing the Iceberg schema
+
+        Examples:
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {
+            ...             "name": "partitions",
+            ...                "type": [
+            ...                "null",
+            ...                {
+            ...                        "type": "array",
+            ...                        "items": {
+            ...                            "type": "record",
+            ...                            "name": "r508",
+            ...                            "fields": [
+            ...                                {
+            ...                                    "name": "contains_null",
+            ...                                    "type": "boolean",
+            ...                                    "doc": "True if any file has a null partition value",
+            ...                                    "field-id": 509,
+            ...                                },
+            ...                                {
+            ...                                    "name": "contains_nan",
+            ...                                    "type": ["null", "boolean"],
+            ...                                    "doc": "True if any file has a nan partition value",
+            ...                                    "default": None,
+            ...                                    "field-id": 518,
+            ...                                },
+            ...                            ],
+            ...                        },
+            ...                        "element-id": 508,
+            ...                    },
+            ...                ],
+            ...                "doc": "Summary for each partition",
+            ...                "default": None,
+            ...                "field-id": 507,
+            ...            },
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=507,
+            ...         name="partitions",
+            ...         field_type=StructType(
+            ...             NestedField(
+            ...                 field_id=509,
+            ...                 name="contains_null",
+            ...                 field_type=BooleanType(),
+            ...                 is_optional=False,
+            ...                 doc="True if any file has a null partition value",
+            ...             ),
+            ...             NestedField(
+            ...                 field_id=518,
+            ...                 name="contains_nan",
+            ...                 field_type=BooleanType(),
+            ...                 is_optional=True,
+            ...                 doc="True if any file has a nan partition value",
+            ...             ),
+            ...         ),
+            ...         is_optional=True,
+            ...         doc="Summary for each partition",
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            field:
+
+        Returns:
+
+        """
+        field_id = field["field-id"]
+        field_name = field["name"]
+        field_doc = field.get("doc")
+
+        avro_type, is_optional = self._resolve_union(field["type"])
+        if isinstance(avro_type, dict):
+            avro_type = avro_type["type"]
+
+        if isinstance(avro_type, str) and avro_type in AvroSchemaConversion.PRIMITIVE_FIELD_TYPE_MAP:
+            return NestedField(
+                field_id=field_id,
+                name=field_name,
+                field_type=AvroSchemaConversion.PRIMITIVE_FIELD_TYPE_MAP[avro_type],
+                is_optional=is_optional,
+                doc=field_doc,
+            )
+        elif avro_type == "record":
+            return NestedField(
+                field_id=field_id, name=field_name, field_type=self._parse_record(field), is_optional=is_optional, doc=field_doc
+            )
+        elif avro_type == "array":
+            inner_type, _ = self._resolve_union(field["type"])
+            assert isinstance(inner_type, dict)
+            inner_type["items"]["field-id"] = inner_type["element-id"]
+            inner_field = self._parse_field(inner_type["items"])
+
+            return NestedField(
+                field_id=field_id, name=field_name, field_type=inner_field.type, is_optional=is_optional, doc=field_doc
+            )
+        elif avro_type == "map":
+            return NestedField(
+                field_id=field_id,
+                name=field_name,
+                field_type=MapType(
+                    key_id=field["key-id"],
+                    key_type=StringType(),
+                    value_id=field["value-id"],
+                    value_type=self._parse_field(field["values"]),

Review Comment:
   I think there are two problems here. First, this returns a field so you probably want to get the `type` from it. Second, `field["values"]` could be a string, which would cause conversion to a field to fail.



-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r872841876


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,274 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Util class for converting between Avro and Iceberg schemas
+
+"""
+import logging
+from typing import Any, Dict, List, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DoubleType,
+    FloatType,
+    IntegerType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+)
+
+logger = logging.getLogger(__name__)
+
+
+class AvroSchemaConversion:
+    PRIMITIVE_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "boolean": BooleanType(),
+        "bytes": BinaryType(),
+        "date": DateType(),
+        "double": DoubleType(),
+        "float": FloatType(),
+        "int": IntegerType(),
+        "long": LongType(),
+        "string": StringType(),
+        "time-millis": TimeType(),
+        "timestamp-millis": TimestampType(),
+    }
+
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expect to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into a Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+
+        Todo:
+            * Implement full support for unions
+            * Implement logical types
+        """
+        fields = self._parse_record(avro_schema)
+        return Schema(*fields.fields, schema_id=1)  # type: ignore
+
+    def _parse_record(self, avro_field: Dict[str, Any]) -> StructType:
+        return StructType(*[self._parse_field(field) for field in avro_field["fields"]])
+
+    def _resolve_union(self, type_union: Union[Dict, List, str]) -> Tuple[Union[str, dict], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        """
+        avro_types: Union[Dict, List]
+        if isinstance(type_union, str):
+            avro_types = [type_union]
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        # Filter the null value, so we know the actual type
+        avro_types = list(filter(lambda t: t != "null", avro_types))
+
+        if len(avro_types) != 1:
+            raise ValueError("Support for unions is yet to be implemented")
+
+        avro_type = avro_types[0]
+
+        return avro_type, is_optional
+
+    def _parse_field(self, field: Dict[str, Any]) -> NestedField:
+        """
+        Recursively walks through the Schema, constructing the Iceberg schema
+
+        Examples:
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {
+            ...             "name": "partitions",
+            ...                "type": [
+            ...                "null",
+            ...                {
+            ...                        "type": "array",
+            ...                        "items": {
+            ...                            "type": "record",
+            ...                            "name": "r508",
+            ...                            "fields": [
+            ...                                {
+            ...                                    "name": "contains_null",
+            ...                                    "type": "boolean",
+            ...                                    "doc": "True if any file has a null partition value",
+            ...                                    "field-id": 509,
+            ...                                },
+            ...                                {
+            ...                                    "name": "contains_nan",
+            ...                                    "type": ["null", "boolean"],
+            ...                                    "doc": "True if any file has a nan partition value",
+            ...                                    "default": None,
+            ...                                    "field-id": 518,
+            ...                                },
+            ...                            ],
+            ...                        },
+            ...                        "element-id": 508,
+            ...                    },
+            ...                ],
+            ...                "doc": "Summary for each partition",
+            ...                "default": None,
+            ...                "field-id": 507,
+            ...            },
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=507,
+            ...         name="partitions",
+            ...         field_type=StructType(
+            ...             NestedField(
+            ...                 field_id=509,
+            ...                 name="contains_null",
+            ...                 field_type=BooleanType(),
+            ...                 is_optional=False,
+            ...                 doc="True if any file has a null partition value",
+            ...             ),
+            ...             NestedField(
+            ...                 field_id=518,
+            ...                 name="contains_nan",
+            ...                 field_type=BooleanType(),
+            ...                 is_optional=True,
+            ...                 doc="True if any file has a nan partition value",
+            ...             ),
+            ...         ),
+            ...         is_optional=True,
+            ...         doc="Summary for each partition",
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            field:
+
+        Returns:
+
+        """
+        field_id = field["field-id"]
+        field_name = field["name"]
+        field_doc = field.get("doc")
+
+        avro_type, is_optional = self._resolve_union(field["type"])
+        if isinstance(avro_type, dict):
+            avro_type = avro_type["type"]
+
+        if isinstance(avro_type, str) and avro_type in AvroSchemaConversion.PRIMITIVE_FIELD_TYPE_MAP:
+            return NestedField(
+                field_id=field_id,
+                name=field_name,
+                field_type=AvroSchemaConversion.PRIMITIVE_FIELD_TYPE_MAP[avro_type],
+                is_optional=is_optional,
+                doc=field_doc,
+            )
+        elif avro_type == "record":
+            return NestedField(
+                field_id=field_id, name=field_name, field_type=self._parse_record(field), is_optional=is_optional, doc=field_doc
+            )
+        elif avro_type == "array":
+            inner_type, _ = self._resolve_union(field["type"])
+            assert isinstance(inner_type, dict)
+            inner_type["items"]["field-id"] = inner_type["element-id"]
+            inner_field = self._parse_field(inner_type["items"])

Review Comment:
   I think it would be cleaner to separate type conversion from field conversion. It's a bit awkward to me that this always converts the type to a field and then gets its type back out. Plus, this hits the same problem as above, where `inner_type["items"]` may not be a dict.



-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r872842611


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,274 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Util class for converting between Avro and Iceberg schemas
+
+"""
+import logging
+from typing import Any, Dict, List, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DoubleType,
+    FloatType,
+    IntegerType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+)
+
+logger = logging.getLogger(__name__)
+
+
+class AvroSchemaConversion:
+    PRIMITIVE_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "boolean": BooleanType(),
+        "bytes": BinaryType(),
+        "date": DateType(),
+        "double": DoubleType(),
+        "float": FloatType(),
+        "int": IntegerType(),
+        "long": LongType(),
+        "string": StringType(),
+        "time-millis": TimeType(),
+        "timestamp-millis": TimestampType(),

Review Comment:
   For date, time, timestamp, and decimal I think you'll need to look at the logical type annotation. For fixed, you'll have to add a branch to the type conversion since you need the length.



-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r883232531


##########
python/src/iceberg/types.py:
##########
@@ -146,7 +146,7 @@ class NestedField(IcebergType):
     name: str = field()
     field_type: IcebergType = field()
     is_optional: bool = field(default=True)
-    doc: Optional[str] = field(default=None, repr=False)
+    doc: Optional[str] = field(default=None, repr=False, compare=False, hash=False)

Review Comment:
   We compare docs in the Java implementation. That's because equality for types is that the type matches exactly, including documentation.
   
   The question about whether a field is "equal" if the doc string is different is the reason why we don't implement `equals` in Java's `Schema` class. We could do the same here if you want.



-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
Fokko commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r884324966


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,451 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Utility class for converting between Avro and Iceberg schemas
+
+"""
+from __future__ import annotations
+
+import logging
+from typing import Any, Dict, List, Tuple
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IcebergType,
+    IntegerType,
+    ListType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+    UUIDType,
+)
+
+logger = logging.getLogger(__name__)
+
+PRIMITIVE_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "boolean": BooleanType(),
+    "bytes": BinaryType(),
+    "double": DoubleType(),
+    "float": FloatType(),
+    "int": IntegerType(),
+    "long": LongType(),
+    "string": StringType(),
+}
+
+LOGICAL_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "date": DateType(),
+    "time-millis": TimeType(),
+    "timestamp-millis": TimestampType(),

Review Comment:
   I've added the mapping, but I think we need to correct the scale when reading the data. Analog to the Java impl: https://github.com/apache/iceberg/blob/90225d6c9413016d611e2ce5eff37db1bc1b4fc5/core/src/main/java/org/apache/iceberg/avro/GenericAvroReader.java#L137-L140



-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r883248868


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,371 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Utility class for converting between Avro and Iceberg schemas
+
+"""
+import logging
+from typing import Any, Dict, List, Optional, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IcebergType,
+    IntegerType,
+    ListType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+    UUIDType,
+)
+
+logger = logging.getLogger(__name__)
+
+
+class AvroSchemaConversion:
+    PRIMITIVE_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "boolean": BooleanType(),
+        "bytes": BinaryType(),
+        "double": DoubleType(),
+        "float": FloatType(),
+        "int": IntegerType(),
+        "long": LongType(),
+        "string": StringType(),
+    }
+
+    LOGICAL_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "date": DateType(),
+        "time-millis": TimeType(),
+        "timestamp-millis": TimestampType(),
+        "uuid": UUIDType(),
+    }
+
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expects to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into a Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+        """
+        fields = self._parse_record(avro_schema)
+        return Schema(*fields.fields, schema_id=1)
+
+    def _parse_record(self, avro_field: Dict[str, Any]) -> StructType:
+        fields = [self._parse_field(field) for field in avro_field["fields"]]
+        return StructType(*fields)  # type: ignore
+
+    def _resolve_union(self, type_union: Union[Dict, List, str]) -> Tuple[Union[str, Dict[str, Any]], bool]:

Review Comment:
   Minor: now that we're using 3.8, we can use the `|` operator for `Union` right?



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r872639302


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,274 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Util class for converting between Avro and Iceberg schemas
+
+"""
+import logging
+from typing import Any, Dict, List, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DoubleType,
+    FloatType,
+    IntegerType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+)
+
+logger = logging.getLogger(__name__)
+
+
+class AvroSchemaConversion:
+    PRIMITIVE_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "boolean": BooleanType(),
+        "bytes": BinaryType(),
+        "date": DateType(),
+        "double": DoubleType(),
+        "float": FloatType(),
+        "int": IntegerType(),
+        "long": LongType(),
+        "string": StringType(),
+        "time-millis": TimeType(),
+        "timestamp-millis": TimestampType(),
+    }
+
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expect to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into a Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+
+        Todo:
+            * Implement full support for unions
+            * Implement logical types
+        """
+        fields = self._parse_record(avro_schema)
+        return Schema(*fields.fields, schema_id=1)  # type: ignore
+
+    def _parse_record(self, avro_field: Dict[str, Any]) -> StructType:
+        return StructType(*[self._parse_field(field) for field in avro_field["fields"]])
+
+    def _resolve_union(self, type_union: Union[Dict, List, str]) -> Tuple[Union[str, dict], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        """
+        avro_types: Union[Dict, List]
+        if isinstance(type_union, str):
+            avro_types = [type_union]

Review Comment:
   Why not just return `(type_union, False)` here directly?



-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r872838083


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,274 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Util class for converting between Avro and Iceberg schemas
+
+"""
+import logging
+from typing import Any, Dict, List, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DoubleType,
+    FloatType,
+    IntegerType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+)
+
+logger = logging.getLogger(__name__)
+
+
+class AvroSchemaConversion:
+    PRIMITIVE_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "boolean": BooleanType(),
+        "bytes": BinaryType(),
+        "date": DateType(),
+        "double": DoubleType(),
+        "float": FloatType(),
+        "int": IntegerType(),
+        "long": LongType(),
+        "string": StringType(),
+        "time-millis": TimeType(),
+        "timestamp-millis": TimestampType(),
+    }
+
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expect to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into a Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+
+        Todo:
+            * Implement full support for unions
+            * Implement logical types
+        """
+        fields = self._parse_record(avro_schema)
+        return Schema(*fields.fields, schema_id=1)  # type: ignore
+
+    def _parse_record(self, avro_field: Dict[str, Any]) -> StructType:
+        return StructType(*[self._parse_field(field) for field in avro_field["fields"]])
+
+    def _resolve_union(self, type_union: Union[Dict, List, str]) -> Tuple[Union[str, dict], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        """
+        avro_types: Union[Dict, List]
+        if isinstance(type_union, str):
+            avro_types = [type_union]
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        # Filter the null value, so we know the actual type
+        avro_types = list(filter(lambda t: t != "null", avro_types))
+
+        if len(avro_types) != 1:
+            raise ValueError("Support for unions is yet to be implemented")
+
+        avro_type = avro_types[0]
+
+        return avro_type, is_optional
+
+    def _parse_field(self, field: Dict[str, Any]) -> NestedField:
+        """
+        Recursively walks through the Schema, constructing the Iceberg schema
+
+        Examples:
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {
+            ...             "name": "partitions",
+            ...                "type": [
+            ...                "null",
+            ...                {
+            ...                        "type": "array",
+            ...                        "items": {
+            ...                            "type": "record",
+            ...                            "name": "r508",
+            ...                            "fields": [
+            ...                                {
+            ...                                    "name": "contains_null",
+            ...                                    "type": "boolean",
+            ...                                    "doc": "True if any file has a null partition value",
+            ...                                    "field-id": 509,
+            ...                                },
+            ...                                {
+            ...                                    "name": "contains_nan",
+            ...                                    "type": ["null", "boolean"],
+            ...                                    "doc": "True if any file has a nan partition value",
+            ...                                    "default": None,
+            ...                                    "field-id": 518,
+            ...                                },
+            ...                            ],
+            ...                        },
+            ...                        "element-id": 508,
+            ...                    },
+            ...                ],
+            ...                "doc": "Summary for each partition",
+            ...                "default": None,
+            ...                "field-id": 507,
+            ...            },
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=507,
+            ...         name="partitions",
+            ...         field_type=StructType(
+            ...             NestedField(
+            ...                 field_id=509,
+            ...                 name="contains_null",
+            ...                 field_type=BooleanType(),
+            ...                 is_optional=False,
+            ...                 doc="True if any file has a null partition value",
+            ...             ),
+            ...             NestedField(
+            ...                 field_id=518,
+            ...                 name="contains_nan",
+            ...                 field_type=BooleanType(),
+            ...                 is_optional=True,
+            ...                 doc="True if any file has a nan partition value",
+            ...             ),
+            ...         ),
+            ...         is_optional=True,
+            ...         doc="Summary for each partition",
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            field:
+
+        Returns:
+
+        """
+        field_id = field["field-id"]
+        field_name = field["name"]
+        field_doc = field.get("doc")
+
+        avro_type, is_optional = self._resolve_union(field["type"])
+        if isinstance(avro_type, dict):
+            avro_type = avro_type["type"]
+
+        if isinstance(avro_type, str) and avro_type in AvroSchemaConversion.PRIMITIVE_FIELD_TYPE_MAP:
+            return NestedField(
+                field_id=field_id,
+                name=field_name,
+                field_type=AvroSchemaConversion.PRIMITIVE_FIELD_TYPE_MAP[avro_type],
+                is_optional=is_optional,
+                doc=field_doc,
+            )
+        elif avro_type == "record":
+            return NestedField(
+                field_id=field_id, name=field_name, field_type=self._parse_record(field), is_optional=is_optional, doc=field_doc
+            )
+        elif avro_type == "array":
+            inner_type, _ = self._resolve_union(field["type"])

Review Comment:
   It seems a little strange to me that this has to call `_resolve_union` twice because the first call replaces `avro_type` with the inner string. Why not keep the original `avro_type` around and use it here rather than calling this a second 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] Fokko commented on a diff in pull request #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
Fokko commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r884315678


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,451 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Utility class for converting between Avro and Iceberg schemas
+
+"""
+from __future__ import annotations
+
+import logging
+from typing import Any, Dict, List, Tuple
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IcebergType,
+    IntegerType,
+    ListType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+    UUIDType,
+)
+
+logger = logging.getLogger(__name__)
+
+PRIMITIVE_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "boolean": BooleanType(),
+    "bytes": BinaryType(),
+    "double": DoubleType(),
+    "float": FloatType(),
+    "int": IntegerType(),
+    "long": LongType(),
+    "string": StringType(),
+}
+
+LOGICAL_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "date": DateType(),
+    "time-millis": TimeType(),
+    "timestamp-millis": TimestampType(),
+    "uuid": UUIDType(),
+}
+
+
+class AvroSchemaConversion:
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expects to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into an Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+        """
+        return Schema(*[self._convert_field(field) for field in avro_schema["fields"]], schema_id=1)
+
+    def _resolve_union(self, type_union: Dict | List | str) -> Tuple[str | Dict[str, Any], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        Raises:
+            TypeError: In the case non-optional union types are encountered
+        """
+        avro_types: Dict | List
+        if isinstance(type_union, str):
+            # It is a primitive and required
+            return type_union, False
+        elif isinstance(type_union, dict):
+            # It is a context and required
+            return type_union, False
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        if len(avro_types) > 2:
+            raise TypeError("Non-optional types aren't part of the Iceberg specification")
+
+        # Filter the null value and return the type
+        return list(filter(lambda t: t != "null", avro_types))[0], is_optional
+
+    def _resolve_inner_type(self, avro_type: Dict[str, Any], inner_field_name: str, id_field: str) -> Tuple[IcebergType, bool]:
+        """
+        Removes any unions inside an array or map, and appends the field-id downstream (if required)
+
+        Args:
+            avro_type: The parent Avro type
+            inner_field_name: For example 'items' in case of an array
+            id_field: Contains the element or key/value id, for example element-id, to be copied to the field-id
+
+        Returns:
+            The inner type and the nullability
+        """
+        plain_type, element_is_optional = self._resolve_union(avro_type[inner_field_name])
+        if isinstance(plain_type, dict):
+            # We need the element-id downstream
+            plain_type["field-id"] = avro_type[id_field]
+        inner_field = self._convert_schema(plain_type)
+        return inner_field, element_is_optional
+
+    def _convert_schema(self, avro_type: str | Dict[str, Any]) -> IcebergType:
+        """
+        Resolves the Avro type
+
+        Args:
+            avro_type: The Avro type, can be simple or complex
+
+        Returns:
+            The equivalent IcebergType
+
+        Raises:
+            ValueError: When there are unknown types
+        """
+        if isinstance(avro_type, str):
+            return PRIMITIVE_FIELD_TYPE_MAPPING[avro_type]
+        elif isinstance(avro_type, dict):
+            if "logicalType" in avro_type:
+                return self._convert_logical_type(avro_type)
+            else:
+                type_identifier = avro_type["type"]
+                if type_identifier == "record":
+                    return self._convert_record_type(avro_type)
+                elif type_identifier == "array":
+                    return self._convert_array_type(avro_type)
+                elif type_identifier == "map":
+                    return self._convert_map_type(avro_type)
+                elif type_identifier == "fixed":
+                    return self._convert_fixed_type(avro_type)
+                else:
+                    raise ValueError(f"Unknown type: {avro_type}")
+        else:
+            raise ValueError(f"Unknown type: {avro_type}")
+
+    def _convert_field(self, field: Dict[str, Any]) -> NestedField:
+        """
+        Converts an Avro field into an Iceberg equivalent field
+        Args:
+            field: The Avro field
+
+        Returns:
+            The Iceberg equivalent field
+        """
+        assert "field-id" in field, "Missing field-id in the Avro field, this is required for converting it to an Iceberg schema"
+
+        plain_type, is_optional = self._resolve_union(field["type"])
+
+        return NestedField(
+            field_id=field["field-id"],
+            name=field["name"],
+            field_type=self._convert_schema(plain_type),
+            is_optional=is_optional,
+            doc=field.get("doc"),
+        )
+
+    def _convert_record_type(self, record_type: Dict[str, Any]) -> StructType:
+        """
+        Converts the fields from a record into an Iceberg struct
+
+        Examples:
+            >>> from iceberg.utils.schema_conversion import AvroSchemaConversion
+            >>> record_type = {
+            ...     "type": "record",
+            ...     "name": "r508",
+            ...     "fields": [{
+            ...         "name": "contains_null",
+            ...         "type": "boolean",
+            ...         "doc": "True if any file has a null partition value",
+            ...         "field-id": 509,
+            ...      }, {
+            ...          "name": "contains_nan",
+            ...          "type": ["null", "boolean"],
+            ...          "doc": "True if any file has a nan partition value",
+            ...          "default": None,
+            ...          "field-id": 518,
+            ...      }],
+            ... }
+            >>> actual = AvroSchemaConversion()._convert_record_type(record_type)
+            >>> expected = StructType(
+            ...     fields=(
+            ...         NestedField(
+            ...             field_id=509,
+            ...             name="contains_null",
+            ...             field_type=BooleanType(),
+            ...             is_optional=False,
+            ...             doc="True if any file has a null partition value",
+            ...         ),
+            ...         NestedField(
+            ...             field_id=518,
+            ...             name="contains_nan",
+            ...             field_type=BooleanType(),
+            ...             is_optional=True,
+            ...             doc="True if any file has a nan partition value",
+            ...         ),
+            ...     )
+            ... )
+            >>> expected == actual
+            True
+
+        Args:
+            record_type: The record type itself
+
+        Returns:
+        """
+        return StructType(*[self._convert_field(field) for field in record_type["fields"]])
+
+    def _convert_array_type(self, array_type: Dict[str, Any]) -> ListType:
+        element_type, element_is_optional = self._resolve_inner_type(array_type, "items", "element-id")
+        return ListType(element_id=array_type["element-id"], element_type=element_type, element_is_optional=element_is_optional)
+
+    def _convert_map_type(self, map_type: Dict[str, Any]) -> MapType:
+        """
+        Args:
+            map_type: The dict that describes the Avro map type
+
+        Examples:
+            >>> from iceberg.utils.schema_conversion import AvroSchemaConversion
+            >>> avro_field = {
+            ...     "type": "map",
+            ...     "values": ["long", "null"],
+            ...     "key-id": 101,
+            ...     "value-id": 102,
+            ... }
+            >>> actual = AvroSchemaConversion()._convert_map_type(avro_field)
+            >>> expected = MapType(
+            ...     key_id=101,
+            ...     key_type=StringType(),
+            ...     value_id=102,
+            ...     value_type=LongType(),
+            ...     value_is_optional=True
+            ... )
+            >>> actual == expected
+            True
+
+        Returns: A MapType
+        """
+        value_type, value_is_optional = self._resolve_union(map_type["values"])
+        return MapType(
+            key_id=map_type["key-id"],
+            # Avro only supports string keys
+            key_type=StringType(),
+            value_id=map_type["value-id"],
+            value_type=self._convert_schema(value_type),
+            value_is_optional=value_is_optional,
+        )
+
+    def _convert_logical_type(self, avro_logical_type: Dict[str, Any]) -> IcebergType:
+        """
+        When a logical type is found, we'll resolve it here. For the decimal and map

Review Comment:
   Good point, 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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
Fokko commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r884315355


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,451 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Utility class for converting between Avro and Iceberg schemas
+
+"""
+from __future__ import annotations
+
+import logging
+from typing import Any, Dict, List, Tuple
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IcebergType,
+    IntegerType,
+    ListType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+    UUIDType,
+)
+
+logger = logging.getLogger(__name__)
+
+PRIMITIVE_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "boolean": BooleanType(),
+    "bytes": BinaryType(),
+    "double": DoubleType(),
+    "float": FloatType(),
+    "int": IntegerType(),
+    "long": LongType(),
+    "string": StringType(),
+}
+
+LOGICAL_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "date": DateType(),
+    "time-millis": TimeType(),
+    "timestamp-millis": TimestampType(),
+    "uuid": UUIDType(),
+}
+
+
+class AvroSchemaConversion:
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expects to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into an Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+        """
+        return Schema(*[self._convert_field(field) for field in avro_schema["fields"]], schema_id=1)
+
+    def _resolve_union(self, type_union: Dict | List | str) -> Tuple[str | Dict[str, Any], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        Raises:
+            TypeError: In the case non-optional union types are encountered
+        """
+        avro_types: Dict | List
+        if isinstance(type_union, str):
+            # It is a primitive and required
+            return type_union, False
+        elif isinstance(type_union, dict):
+            # It is a context and required
+            return type_union, False
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        if len(avro_types) > 2:
+            raise TypeError("Non-optional types aren't part of the Iceberg specification")
+
+        # Filter the null value and return the type
+        return list(filter(lambda t: t != "null", avro_types))[0], is_optional
+
+    def _resolve_inner_type(self, avro_type: Dict[str, Any], inner_field_name: str, id_field: str) -> Tuple[IcebergType, bool]:

Review Comment:
   Good one, moved it 👍🏻 



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r884297408


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,451 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Utility class for converting between Avro and Iceberg schemas
+
+"""
+from __future__ import annotations
+
+import logging
+from typing import Any, Dict, List, Tuple
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IcebergType,
+    IntegerType,
+    ListType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+    UUIDType,
+)
+
+logger = logging.getLogger(__name__)
+
+PRIMITIVE_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "boolean": BooleanType(),
+    "bytes": BinaryType(),
+    "double": DoubleType(),
+    "float": FloatType(),
+    "int": IntegerType(),
+    "long": LongType(),
+    "string": StringType(),
+}
+
+LOGICAL_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "date": DateType(),
+    "time-millis": TimeType(),
+    "timestamp-millis": TimestampType(),
+    "uuid": UUIDType(),
+}
+
+
+class AvroSchemaConversion:
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expects to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into an Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+        """
+        return Schema(*[self._convert_field(field) for field in avro_schema["fields"]], schema_id=1)
+
+    def _resolve_union(self, type_union: Dict | List | str) -> Tuple[str | Dict[str, Any], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        Raises:
+            TypeError: In the case non-optional union types are encountered
+        """
+        avro_types: Dict | List
+        if isinstance(type_union, str):
+            # It is a primitive and required
+            return type_union, False
+        elif isinstance(type_union, dict):
+            # It is a context and required
+            return type_union, False
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        if len(avro_types) > 2:
+            raise TypeError("Non-optional types aren't part of the Iceberg specification")
+
+        # Filter the null value and return the type
+        return list(filter(lambda t: t != "null", avro_types))[0], is_optional
+
+    def _resolve_inner_type(self, avro_type: Dict[str, Any], inner_field_name: str, id_field: str) -> Tuple[IcebergType, bool]:
+        """
+        Removes any unions inside an array or map, and appends the field-id downstream (if required)
+
+        Args:
+            avro_type: The parent Avro type
+            inner_field_name: For example 'items' in case of an array
+            id_field: Contains the element or key/value id, for example element-id, to be copied to the field-id
+
+        Returns:
+            The inner type and the nullability
+        """
+        plain_type, element_is_optional = self._resolve_union(avro_type[inner_field_name])
+        if isinstance(plain_type, dict):
+            # We need the element-id downstream
+            plain_type["field-id"] = avro_type[id_field]
+        inner_field = self._convert_schema(plain_type)
+        return inner_field, element_is_optional
+
+    def _convert_schema(self, avro_type: str | Dict[str, Any]) -> IcebergType:
+        """
+        Resolves the Avro type
+
+        Args:
+            avro_type: The Avro type, can be simple or complex
+
+        Returns:
+            The equivalent IcebergType
+
+        Raises:
+            ValueError: When there are unknown types
+        """
+        if isinstance(avro_type, str):
+            return PRIMITIVE_FIELD_TYPE_MAPPING[avro_type]
+        elif isinstance(avro_type, dict):
+            if "logicalType" in avro_type:
+                return self._convert_logical_type(avro_type)
+            else:
+                type_identifier = avro_type["type"]
+                if type_identifier == "record":
+                    return self._convert_record_type(avro_type)
+                elif type_identifier == "array":
+                    return self._convert_array_type(avro_type)
+                elif type_identifier == "map":
+                    return self._convert_map_type(avro_type)
+                elif type_identifier == "fixed":
+                    return self._convert_fixed_type(avro_type)
+                else:
+                    raise ValueError(f"Unknown type: {avro_type}")
+        else:
+            raise ValueError(f"Unknown type: {avro_type}")
+
+    def _convert_field(self, field: Dict[str, Any]) -> NestedField:
+        """
+        Converts an Avro field into an Iceberg equivalent field
+        Args:
+            field: The Avro field
+
+        Returns:
+            The Iceberg equivalent field
+        """
+        assert "field-id" in field, "Missing field-id in the Avro field, this is required for converting it to an Iceberg schema"
+
+        plain_type, is_optional = self._resolve_union(field["type"])
+
+        return NestedField(
+            field_id=field["field-id"],
+            name=field["name"],
+            field_type=self._convert_schema(plain_type),
+            is_optional=is_optional,
+            doc=field.get("doc"),
+        )
+
+    def _convert_record_type(self, record_type: Dict[str, Any]) -> StructType:
+        """
+        Converts the fields from a record into an Iceberg struct
+
+        Examples:
+            >>> from iceberg.utils.schema_conversion import AvroSchemaConversion
+            >>> record_type = {
+            ...     "type": "record",
+            ...     "name": "r508",
+            ...     "fields": [{
+            ...         "name": "contains_null",
+            ...         "type": "boolean",
+            ...         "doc": "True if any file has a null partition value",
+            ...         "field-id": 509,
+            ...      }, {
+            ...          "name": "contains_nan",
+            ...          "type": ["null", "boolean"],
+            ...          "doc": "True if any file has a nan partition value",
+            ...          "default": None,
+            ...          "field-id": 518,
+            ...      }],
+            ... }
+            >>> actual = AvroSchemaConversion()._convert_record_type(record_type)
+            >>> expected = StructType(
+            ...     fields=(
+            ...         NestedField(
+            ...             field_id=509,
+            ...             name="contains_null",
+            ...             field_type=BooleanType(),
+            ...             is_optional=False,
+            ...             doc="True if any file has a null partition value",
+            ...         ),
+            ...         NestedField(
+            ...             field_id=518,
+            ...             name="contains_nan",
+            ...             field_type=BooleanType(),
+            ...             is_optional=True,
+            ...             doc="True if any file has a nan partition value",
+            ...         ),
+            ...     )
+            ... )
+            >>> expected == actual
+            True
+
+        Args:
+            record_type: The record type itself
+
+        Returns:
+        """
+        return StructType(*[self._convert_field(field) for field in record_type["fields"]])
+
+    def _convert_array_type(self, array_type: Dict[str, Any]) -> ListType:
+        element_type, element_is_optional = self._resolve_inner_type(array_type, "items", "element-id")
+        return ListType(element_id=array_type["element-id"], element_type=element_type, element_is_optional=element_is_optional)
+
+    def _convert_map_type(self, map_type: Dict[str, Any]) -> MapType:
+        """
+        Args:
+            map_type: The dict that describes the Avro map type
+
+        Examples:
+            >>> from iceberg.utils.schema_conversion import AvroSchemaConversion
+            >>> avro_field = {
+            ...     "type": "map",
+            ...     "values": ["long", "null"],
+            ...     "key-id": 101,
+            ...     "value-id": 102,
+            ... }
+            >>> actual = AvroSchemaConversion()._convert_map_type(avro_field)
+            >>> expected = MapType(
+            ...     key_id=101,
+            ...     key_type=StringType(),
+            ...     value_id=102,
+            ...     value_type=LongType(),
+            ...     value_is_optional=True
+            ... )
+            >>> actual == expected
+            True
+
+        Returns: A MapType
+        """
+        value_type, value_is_optional = self._resolve_union(map_type["values"])
+        return MapType(
+            key_id=map_type["key-id"],
+            # Avro only supports string keys
+            key_type=StringType(),
+            value_id=map_type["value-id"],
+            value_type=self._convert_schema(value_type),
+            value_is_optional=value_is_optional,
+        )
+
+    def _convert_logical_type(self, avro_logical_type: Dict[str, Any]) -> IcebergType:
+        """
+        When a logical type is found, we'll resolve it here. For the decimal and map

Review Comment:
   Style: Avoid the use of personal pronouns, like "I" and "we". Those make docs longer and don't add clarity. Here, you can use "Convert a schema with a logical type annotation", which is more direct.



-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r884325933


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,451 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Utility class for converting between Avro and Iceberg schemas
+
+"""
+from __future__ import annotations
+
+import logging
+from typing import Any, Dict, List, Tuple
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IcebergType,
+    IntegerType,
+    ListType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+    UUIDType,
+)
+
+logger = logging.getLogger(__name__)
+
+PRIMITIVE_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "boolean": BooleanType(),
+    "bytes": BinaryType(),
+    "double": DoubleType(),
+    "float": FloatType(),
+    "int": IntegerType(),
+    "long": LongType(),
+    "string": StringType(),
+}
+
+LOGICAL_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "date": DateType(),
+    "time-millis": TimeType(),
+    "timestamp-millis": TimestampType(),

Review Comment:
   Yes, exactly. We'll need to be aware of whether it is in millis or micros when reading the file!



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
Fokko commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r877579114


##########
python/src/iceberg/types.py:
##########
@@ -146,7 +146,7 @@ class NestedField(IcebergType):
     name: str = field()
     field_type: IcebergType = field()
     is_optional: bool = field(default=True)
-    doc: Optional[str] = field(default=None, repr=False)
+    doc: Optional[str] = field(default=None, repr=False, compare=False, hash=False)

Review Comment:
   @rdblue do we want to compare the docs as well? I know that for Avro we only compare the fields that are part of the canonical representation. I caught this in a test where my example didn't have the docs 🤔 



-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r884295152


##########
python/src/iceberg/schema.py:
##########
@@ -62,8 +62,23 @@ def __repr__(self):
             f"Schema(fields={repr(self.columns)}, schema_id={self.schema_id}, identifier_field_ids={self.identifier_field_ids})"
         )
 
+    def __eq__(self, other) -> bool:
+        if not other:
+            return False
+
+        if not isinstance(other, Schema):
+            return False
+
+        if len(self.columns) != len(other.columns):
+            return False
+
+        identifier_field_ids_is_equal = self.identifier_field_ids == other.identifier_field_ids
+        schema_is_equal = all([lhs == rhs for lhs, rhs in zip(self.columns, other.columns)])
+
+        return identifier_field_ids_is_equal and schema_is_equal
+
     @property
-    def columns(self) -> Iterable[NestedField]:
+    def columns(self) -> Tuple[NestedField]:

Review Comment:
   Should this be `Tuple[NestedField, ...]` since it isn't a single field in the tuple?
   
   From the [typing.Tuple docs](https://docs.python.org/3/library/typing.html#typing.Tuple):
   > To specify a variable-length tuple of homogeneous type, use literal ellipsis, e.g. Tuple[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] Fokko commented on a diff in pull request #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
Fokko commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r884324676


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,451 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Utility class for converting between Avro and Iceberg schemas
+
+"""
+from __future__ import annotations
+
+import logging
+from typing import Any, Dict, List, Tuple
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IcebergType,
+    IntegerType,
+    ListType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+    UUIDType,
+)
+
+logger = logging.getLogger(__name__)
+
+PRIMITIVE_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "boolean": BooleanType(),
+    "bytes": BinaryType(),
+    "double": DoubleType(),
+    "float": FloatType(),
+    "int": IntegerType(),
+    "long": LongType(),
+    "string": StringType(),
+}
+
+LOGICAL_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "date": DateType(),
+    "time-millis": TimeType(),
+    "timestamp-millis": TimestampType(),
+    "uuid": UUIDType(),
+}
+
+
+class AvroSchemaConversion:
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expects to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into an Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+        """
+        return Schema(*[self._convert_field(field) for field in avro_schema["fields"]], schema_id=1)
+
+    def _resolve_union(self, type_union: Dict | List | str) -> Tuple[str | Dict[str, Any], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        Raises:
+            TypeError: In the case non-optional union types are encountered
+        """
+        avro_types: Dict | List
+        if isinstance(type_union, str):
+            # It is a primitive and required
+            return type_union, False
+        elif isinstance(type_union, dict):
+            # It is a context and required
+            return type_union, False
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        if len(avro_types) > 2:
+            raise TypeError("Non-optional types aren't part of the Iceberg specification")
+
+        # Filter the null value and return the type
+        return list(filter(lambda t: t != "null", avro_types))[0], is_optional
+
+    def _resolve_inner_type(self, avro_type: Dict[str, Any], inner_field_name: str, id_field: str) -> Tuple[IcebergType, bool]:
+        """
+        Removes any unions inside an array or map, and appends the field-id downstream (if required)
+
+        Args:
+            avro_type: The parent Avro type
+            inner_field_name: For example 'items' in case of an array
+            id_field: Contains the element or key/value id, for example element-id, to be copied to the field-id
+
+        Returns:
+            The inner type and the nullability
+        """
+        plain_type, element_is_optional = self._resolve_union(avro_type[inner_field_name])
+        if isinstance(plain_type, dict):
+            # We need the element-id downstream
+            plain_type["field-id"] = avro_type[id_field]

Review Comment:
   Hey Ryan, I agree. I don't think we really need the method at all. I've removed it and just resolved the type instead of the field. Still some old code dangling around.



-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
Fokko commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r876286780


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,274 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Util class for converting between Avro and Iceberg schemas
+
+"""
+import logging
+from typing import Any, Dict, List, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DoubleType,
+    FloatType,
+    IntegerType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+)
+
+logger = logging.getLogger(__name__)
+
+
+class AvroSchemaConversion:
+    PRIMITIVE_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "boolean": BooleanType(),
+        "bytes": BinaryType(),
+        "date": DateType(),
+        "double": DoubleType(),
+        "float": FloatType(),
+        "int": IntegerType(),
+        "long": LongType(),
+        "string": StringType(),
+        "time-millis": TimeType(),
+        "timestamp-millis": TimestampType(),
+    }
+
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expect to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into a Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+
+        Todo:
+            * Implement full support for unions
+            * Implement logical types
+        """
+        fields = self._parse_record(avro_schema)
+        return Schema(*fields.fields, schema_id=1)  # type: ignore
+
+    def _parse_record(self, avro_field: Dict[str, Any]) -> StructType:
+        return StructType(*[self._parse_field(field) for field in avro_field["fields"]])
+
+    def _resolve_union(self, type_union: Union[Dict, List, str]) -> Tuple[Union[str, dict], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        """
+        avro_types: Union[Dict, List]
+        if isinstance(type_union, str):
+            avro_types = [type_union]
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        # Filter the null value, so we know the actual type
+        avro_types = list(filter(lambda t: t != "null", avro_types))
+
+        if len(avro_types) != 1:
+            raise ValueError("Support for unions is yet to be implemented")
+
+        avro_type = avro_types[0]
+
+        return avro_type, is_optional
+
+    def _parse_field(self, field: Dict[str, Any]) -> NestedField:
+        """
+        Recursively walks through the Schema, constructing the Iceberg schema
+
+        Examples:
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {
+            ...             "name": "partitions",
+            ...                "type": [
+            ...                "null",
+            ...                {
+            ...                        "type": "array",
+            ...                        "items": {
+            ...                            "type": "record",
+            ...                            "name": "r508",
+            ...                            "fields": [
+            ...                                {
+            ...                                    "name": "contains_null",
+            ...                                    "type": "boolean",
+            ...                                    "doc": "True if any file has a null partition value",
+            ...                                    "field-id": 509,
+            ...                                },
+            ...                                {
+            ...                                    "name": "contains_nan",
+            ...                                    "type": ["null", "boolean"],
+            ...                                    "doc": "True if any file has a nan partition value",
+            ...                                    "default": None,
+            ...                                    "field-id": 518,
+            ...                                },
+            ...                            ],
+            ...                        },
+            ...                        "element-id": 508,
+            ...                    },
+            ...                ],
+            ...                "doc": "Summary for each partition",
+            ...                "default": None,
+            ...                "field-id": 507,
+            ...            },
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=507,
+            ...         name="partitions",
+            ...         field_type=StructType(
+            ...             NestedField(
+            ...                 field_id=509,
+            ...                 name="contains_null",
+            ...                 field_type=BooleanType(),
+            ...                 is_optional=False,
+            ...                 doc="True if any file has a null partition value",
+            ...             ),
+            ...             NestedField(
+            ...                 field_id=518,
+            ...                 name="contains_nan",
+            ...                 field_type=BooleanType(),
+            ...                 is_optional=True,
+            ...                 doc="True if any file has a nan partition value",
+            ...             ),
+            ...         ),
+            ...         is_optional=True,
+            ...         doc="Summary for each partition",
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            field:
+
+        Returns:
+
+        """
+        field_id = field["field-id"]
+        field_name = field["name"]
+        field_doc = field.get("doc")
+
+        avro_type, is_optional = self._resolve_union(field["type"])
+        if isinstance(avro_type, dict):
+            avro_type = avro_type["type"]
+
+        if isinstance(avro_type, str) and avro_type in AvroSchemaConversion.PRIMITIVE_FIELD_TYPE_MAP:
+            return NestedField(
+                field_id=field_id,
+                name=field_name,
+                field_type=AvroSchemaConversion.PRIMITIVE_FIELD_TYPE_MAP[avro_type],
+                is_optional=is_optional,
+                doc=field_doc,
+            )
+        elif avro_type == "record":
+            return NestedField(
+                field_id=field_id, name=field_name, field_type=self._parse_record(field), is_optional=is_optional, doc=field_doc
+            )
+        elif avro_type == "array":
+            inner_type, _ = self._resolve_union(field["type"])

Review Comment:
   Ah, this is a relic from the refactoring. Thanks for catching 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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
Fokko commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r876403572


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,274 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Util class for converting between Avro and Iceberg schemas
+
+"""
+import logging
+from typing import Any, Dict, List, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DoubleType,
+    FloatType,
+    IntegerType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+)
+
+logger = logging.getLogger(__name__)
+
+
+class AvroSchemaConversion:
+    PRIMITIVE_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "boolean": BooleanType(),
+        "bytes": BinaryType(),
+        "date": DateType(),
+        "double": DoubleType(),
+        "float": FloatType(),
+        "int": IntegerType(),
+        "long": LongType(),
+        "string": StringType(),
+        "time-millis": TimeType(),
+        "timestamp-millis": TimestampType(),
+    }
+
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expect to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into a Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+
+        Todo:
+            * Implement full support for unions
+            * Implement logical types
+        """
+        fields = self._parse_record(avro_schema)
+        return Schema(*fields.fields, schema_id=1)  # type: ignore
+
+    def _parse_record(self, avro_field: Dict[str, Any]) -> StructType:
+        return StructType(*[self._parse_field(field) for field in avro_field["fields"]])
+
+    def _resolve_union(self, type_union: Union[Dict, List, str]) -> Tuple[Union[str, dict], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        """
+        avro_types: Union[Dict, List]
+        if isinstance(type_union, str):
+            avro_types = [type_union]
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        # Filter the null value, so we know the actual type
+        avro_types = list(filter(lambda t: t != "null", avro_types))
+
+        if len(avro_types) != 1:
+            raise ValueError("Support for unions is yet to be implemented")
+
+        avro_type = avro_types[0]
+
+        return avro_type, is_optional
+
+    def _parse_field(self, field: Dict[str, Any]) -> NestedField:
+        """
+        Recursively walks through the Schema, constructing the Iceberg schema
+
+        Examples:
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {
+            ...             "name": "partitions",
+            ...                "type": [
+            ...                "null",
+            ...                {
+            ...                        "type": "array",
+            ...                        "items": {
+            ...                            "type": "record",
+            ...                            "name": "r508",
+            ...                            "fields": [
+            ...                                {
+            ...                                    "name": "contains_null",
+            ...                                    "type": "boolean",
+            ...                                    "doc": "True if any file has a null partition value",
+            ...                                    "field-id": 509,
+            ...                                },
+            ...                                {
+            ...                                    "name": "contains_nan",
+            ...                                    "type": ["null", "boolean"],
+            ...                                    "doc": "True if any file has a nan partition value",
+            ...                                    "default": None,
+            ...                                    "field-id": 518,
+            ...                                },
+            ...                            ],
+            ...                        },
+            ...                        "element-id": 508,
+            ...                    },
+            ...                ],
+            ...                "doc": "Summary for each partition",
+            ...                "default": None,
+            ...                "field-id": 507,
+            ...            },
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=507,
+            ...         name="partitions",
+            ...         field_type=StructType(
+            ...             NestedField(
+            ...                 field_id=509,
+            ...                 name="contains_null",
+            ...                 field_type=BooleanType(),
+            ...                 is_optional=False,
+            ...                 doc="True if any file has a null partition value",
+            ...             ),
+            ...             NestedField(
+            ...                 field_id=518,
+            ...                 name="contains_nan",
+            ...                 field_type=BooleanType(),
+            ...                 is_optional=True,
+            ...                 doc="True if any file has a nan partition value",
+            ...             ),
+            ...         ),
+            ...         is_optional=True,
+            ...         doc="Summary for each partition",
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            field:
+
+        Returns:
+
+        """
+        field_id = field["field-id"]
+        field_name = field["name"]
+        field_doc = field.get("doc")
+
+        avro_type, is_optional = self._resolve_union(field["type"])
+        if isinstance(avro_type, dict):
+            avro_type = avro_type["type"]
+
+        if isinstance(avro_type, str) and avro_type in AvroSchemaConversion.PRIMITIVE_FIELD_TYPE_MAP:
+            return NestedField(
+                field_id=field_id,
+                name=field_name,
+                field_type=AvroSchemaConversion.PRIMITIVE_FIELD_TYPE_MAP[avro_type],
+                is_optional=is_optional,
+                doc=field_doc,
+            )
+        elif avro_type == "record":
+            return NestedField(
+                field_id=field_id, name=field_name, field_type=self._parse_record(field), is_optional=is_optional, doc=field_doc
+            )
+        elif avro_type == "array":
+            inner_type, _ = self._resolve_union(field["type"])
+            assert isinstance(inner_type, dict)
+            inner_type["items"]["field-id"] = inner_type["element-id"]

Review Comment:
   That's correct, it can also be a primitive 👍🏻  I've created test cases and working on a fix 



-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
Fokko commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r883878678


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,371 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Utility class for converting between Avro and Iceberg schemas
+
+"""
+import logging
+from typing import Any, Dict, List, Optional, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IcebergType,
+    IntegerType,
+    ListType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+    UUIDType,
+)
+
+logger = logging.getLogger(__name__)
+
+
+class AvroSchemaConversion:
+    PRIMITIVE_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "boolean": BooleanType(),
+        "bytes": BinaryType(),
+        "double": DoubleType(),
+        "float": FloatType(),
+        "int": IntegerType(),
+        "long": LongType(),
+        "string": StringType(),
+    }
+
+    LOGICAL_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "date": DateType(),
+        "time-millis": TimeType(),
+        "timestamp-millis": TimestampType(),
+        "uuid": UUIDType(),
+    }
+
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expects to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into a Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+        """
+        fields = self._parse_record(avro_schema)
+        return Schema(*fields.fields, schema_id=1)
+
+    def _parse_record(self, avro_field: Dict[str, Any]) -> StructType:

Review Comment:
   Good one, 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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
Fokko commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r883876408


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,371 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Utility class for converting between Avro and Iceberg schemas
+
+"""
+import logging
+from typing import Any, Dict, List, Optional, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IcebergType,
+    IntegerType,
+    ListType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+    UUIDType,
+)
+
+logger = logging.getLogger(__name__)
+
+
+class AvroSchemaConversion:
+    PRIMITIVE_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "boolean": BooleanType(),
+        "bytes": BinaryType(),
+        "double": DoubleType(),
+        "float": FloatType(),
+        "int": IntegerType(),
+        "long": LongType(),
+        "string": StringType(),
+    }
+
+    LOGICAL_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "date": DateType(),
+        "time-millis": TimeType(),
+        "timestamp-millis": TimestampType(),
+        "uuid": UUIDType(),
+    }
+
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expects to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into a Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+        """
+        fields = self._parse_record(avro_schema)
+        return Schema(*fields.fields, schema_id=1)
+
+    def _parse_record(self, avro_field: Dict[str, Any]) -> StructType:
+        fields = [self._parse_field(field) for field in avro_field["fields"]]
+        return StructType(*fields)  # type: ignore
+
+    def _resolve_union(self, type_union: Union[Dict, List, str]) -> Tuple[Union[str, Dict[str, Any]], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        Raises:
+            TypeError: In the case non-optional union types are encountered
+        """
+        avro_types: Union[Dict, List]
+        if isinstance(type_union, str):
+            # It is a primitive and required
+            return type_union, False
+        elif isinstance(type_union, dict):
+            # It is a context and required
+            return type_union, False
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types

Review Comment:
   But then a `['string', 'bytes']` union could also slip through



-- 
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] kbendick commented on a diff in pull request #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r869496501


##########
python/src/iceberg/schema.py:
##########
@@ -67,6 +67,17 @@ def __repr__(self):
             f"Schema(fields={repr(self.columns)}, schema_id={self.schema_id}, identifier_field_ids={self.identifier_field_ids})"
         )
 
+    def __eq__(self, other) -> bool:

Review Comment:
   I think we also need to check the identifier fields. Java's `Schema::isSameSchema` does so.
   
   https://github.com/apache/iceberg/blob/06cdfa3f7c4ebd26766bc67100ba5d30b6bf3940/api/src/main/java/org/apache/iceberg/Schema.java#L444-L452



-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r872643178


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,274 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Util class for converting between Avro and Iceberg schemas
+
+"""
+import logging
+from typing import Any, Dict, List, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DoubleType,
+    FloatType,
+    IntegerType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+)
+
+logger = logging.getLogger(__name__)
+
+
+class AvroSchemaConversion:
+    PRIMITIVE_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "boolean": BooleanType(),
+        "bytes": BinaryType(),
+        "date": DateType(),
+        "double": DoubleType(),
+        "float": FloatType(),
+        "int": IntegerType(),
+        "long": LongType(),
+        "string": StringType(),
+        "time-millis": TimeType(),
+        "timestamp-millis": TimestampType(),
+    }
+
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expect to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into a Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+
+        Todo:
+            * Implement full support for unions
+            * Implement logical types
+        """
+        fields = self._parse_record(avro_schema)
+        return Schema(*fields.fields, schema_id=1)  # type: ignore
+
+    def _parse_record(self, avro_field: Dict[str, Any]) -> StructType:
+        return StructType(*[self._parse_field(field) for field in avro_field["fields"]])
+
+    def _resolve_union(self, type_union: Union[Dict, List, str]) -> Tuple[Union[str, dict], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        """
+        avro_types: Union[Dict, List]
+        if isinstance(type_union, str):
+            avro_types = [type_union]
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        # Filter the null value, so we know the actual type
+        avro_types = list(filter(lambda t: t != "null", avro_types))
+
+        if len(avro_types) != 1:
+            raise ValueError("Support for unions is yet to be implemented")
+
+        avro_type = avro_types[0]
+
+        return avro_type, is_optional
+
+    def _parse_field(self, field: Dict[str, Any]) -> NestedField:
+        """
+        Recursively walks through the Schema, constructing the Iceberg schema
+
+        Examples:
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {
+            ...             "name": "partitions",
+            ...                "type": [
+            ...                "null",
+            ...                {
+            ...                        "type": "array",
+            ...                        "items": {
+            ...                            "type": "record",
+            ...                            "name": "r508",
+            ...                            "fields": [
+            ...                                {
+            ...                                    "name": "contains_null",
+            ...                                    "type": "boolean",
+            ...                                    "doc": "True if any file has a null partition value",
+            ...                                    "field-id": 509,
+            ...                                },
+            ...                                {
+            ...                                    "name": "contains_nan",
+            ...                                    "type": ["null", "boolean"],
+            ...                                    "doc": "True if any file has a nan partition value",
+            ...                                    "default": None,
+            ...                                    "field-id": 518,
+            ...                                },
+            ...                            ],
+            ...                        },
+            ...                        "element-id": 508,
+            ...                    },
+            ...                ],
+            ...                "doc": "Summary for each partition",
+            ...                "default": None,
+            ...                "field-id": 507,
+            ...            },
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=507,
+            ...         name="partitions",
+            ...         field_type=StructType(
+            ...             NestedField(
+            ...                 field_id=509,
+            ...                 name="contains_null",
+            ...                 field_type=BooleanType(),
+            ...                 is_optional=False,
+            ...                 doc="True if any file has a null partition value",
+            ...             ),
+            ...             NestedField(
+            ...                 field_id=518,
+            ...                 name="contains_nan",
+            ...                 field_type=BooleanType(),
+            ...                 is_optional=True,
+            ...                 doc="True if any file has a nan partition value",
+            ...             ),
+            ...         ),
+            ...         is_optional=True,
+            ...         doc="Summary for each partition",
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            field:
+
+        Returns:
+
+        """
+        field_id = field["field-id"]

Review Comment:
   Should we throw a better error than `KeyError` if there is no field ID? I think that would be nice since this ID is specific to Iceberg.



-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
Fokko commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r876404446


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,274 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Util class for converting between Avro and Iceberg schemas
+
+"""
+import logging
+from typing import Any, Dict, List, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DoubleType,
+    FloatType,
+    IntegerType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+)
+
+logger = logging.getLogger(__name__)
+
+
+class AvroSchemaConversion:
+    PRIMITIVE_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "boolean": BooleanType(),
+        "bytes": BinaryType(),
+        "date": DateType(),
+        "double": DoubleType(),
+        "float": FloatType(),
+        "int": IntegerType(),
+        "long": LongType(),
+        "string": StringType(),
+        "time-millis": TimeType(),
+        "timestamp-millis": TimestampType(),
+    }
+
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expect to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into a Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+
+        Todo:
+            * Implement full support for unions
+            * Implement logical types
+        """
+        fields = self._parse_record(avro_schema)
+        return Schema(*fields.fields, schema_id=1)  # type: ignore
+
+    def _parse_record(self, avro_field: Dict[str, Any]) -> StructType:
+        return StructType(*[self._parse_field(field) for field in avro_field["fields"]])
+
+    def _resolve_union(self, type_union: Union[Dict, List, str]) -> Tuple[Union[str, dict], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        """
+        avro_types: Union[Dict, List]
+        if isinstance(type_union, str):
+            avro_types = [type_union]
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        # Filter the null value, so we know the actual type
+        avro_types = list(filter(lambda t: t != "null", avro_types))
+
+        if len(avro_types) != 1:
+            raise ValueError("Support for unions is yet to be implemented")
+
+        avro_type = avro_types[0]
+
+        return avro_type, is_optional
+
+    def _parse_field(self, field: Dict[str, Any]) -> NestedField:
+        """
+        Recursively walks through the Schema, constructing the Iceberg schema
+
+        Examples:
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {
+            ...             "name": "partitions",
+            ...                "type": [
+            ...                "null",
+            ...                {
+            ...                        "type": "array",
+            ...                        "items": {
+            ...                            "type": "record",
+            ...                            "name": "r508",
+            ...                            "fields": [
+            ...                                {
+            ...                                    "name": "contains_null",
+            ...                                    "type": "boolean",
+            ...                                    "doc": "True if any file has a null partition value",
+            ...                                    "field-id": 509,
+            ...                                },
+            ...                                {
+            ...                                    "name": "contains_nan",
+            ...                                    "type": ["null", "boolean"],
+            ...                                    "doc": "True if any file has a nan partition value",
+            ...                                    "default": None,
+            ...                                    "field-id": 518,
+            ...                                },
+            ...                            ],
+            ...                        },
+            ...                        "element-id": 508,
+            ...                    },
+            ...                ],
+            ...                "doc": "Summary for each partition",
+            ...                "default": None,
+            ...                "field-id": 507,
+            ...            },
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=507,
+            ...         name="partitions",
+            ...         field_type=StructType(
+            ...             NestedField(
+            ...                 field_id=509,
+            ...                 name="contains_null",
+            ...                 field_type=BooleanType(),
+            ...                 is_optional=False,
+            ...                 doc="True if any file has a null partition value",
+            ...             ),
+            ...             NestedField(
+            ...                 field_id=518,
+            ...                 name="contains_nan",
+            ...                 field_type=BooleanType(),
+            ...                 is_optional=True,
+            ...                 doc="True if any file has a nan partition value",
+            ...             ),
+            ...         ),
+            ...         is_optional=True,
+            ...         doc="Summary for each partition",
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            field:
+
+        Returns:
+
+        """
+        field_id = field["field-id"]
+        field_name = field["name"]
+        field_doc = field.get("doc")
+
+        avro_type, is_optional = self._resolve_union(field["type"])
+        if isinstance(avro_type, dict):
+            avro_type = avro_type["type"]
+
+        if isinstance(avro_type, str) and avro_type in AvroSchemaConversion.PRIMITIVE_FIELD_TYPE_MAP:
+            return NestedField(
+                field_id=field_id,
+                name=field_name,
+                field_type=AvroSchemaConversion.PRIMITIVE_FIELD_TYPE_MAP[avro_type],
+                is_optional=is_optional,
+                doc=field_doc,
+            )
+        elif avro_type == "record":
+            return NestedField(
+                field_id=field_id, name=field_name, field_type=self._parse_record(field), is_optional=is_optional, doc=field_doc
+            )
+        elif avro_type == "array":
+            inner_type, _ = self._resolve_union(field["type"])
+            assert isinstance(inner_type, dict)
+            inner_type["items"]["field-id"] = inner_type["element-id"]
+            inner_field = self._parse_field(inner_type["items"])

Review Comment:
   Thanks for pointing this out. There is actually a bug in the array conversion. In the test cases, you can see that the ListType is missing, I'll push a fix tomorrow morning.



-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r884296225


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,451 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Utility class for converting between Avro and Iceberg schemas
+
+"""
+from __future__ import annotations
+
+import logging
+from typing import Any, Dict, List, Tuple
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IcebergType,
+    IntegerType,
+    ListType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+    UUIDType,
+)
+
+logger = logging.getLogger(__name__)
+
+PRIMITIVE_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "boolean": BooleanType(),
+    "bytes": BinaryType(),
+    "double": DoubleType(),
+    "float": FloatType(),
+    "int": IntegerType(),
+    "long": LongType(),
+    "string": StringType(),
+}
+
+LOGICAL_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "date": DateType(),
+    "time-millis": TimeType(),
+    "timestamp-millis": TimestampType(),
+    "uuid": UUIDType(),
+}
+
+
+class AvroSchemaConversion:
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expects to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into an Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+        """
+        return Schema(*[self._convert_field(field) for field in avro_schema["fields"]], schema_id=1)
+
+    def _resolve_union(self, type_union: Dict | List | str) -> Tuple[str | Dict[str, Any], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        Raises:
+            TypeError: In the case non-optional union types are encountered
+        """
+        avro_types: Dict | List
+        if isinstance(type_union, str):
+            # It is a primitive and required
+            return type_union, False
+        elif isinstance(type_union, dict):
+            # It is a context and required
+            return type_union, False
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        if len(avro_types) > 2:
+            raise TypeError("Non-optional types aren't part of the Iceberg specification")
+
+        # Filter the null value and return the type
+        return list(filter(lambda t: t != "null", avro_types))[0], is_optional
+
+    def _resolve_inner_type(self, avro_type: Dict[str, Any], inner_field_name: str, id_field: str) -> Tuple[IcebergType, bool]:
+        """
+        Removes any unions inside an array or map, and appends the field-id downstream (if required)
+
+        Args:
+            avro_type: The parent Avro type
+            inner_field_name: For example 'items' in case of an array
+            id_field: Contains the element or key/value id, for example element-id, to be copied to the field-id
+
+        Returns:
+            The inner type and the nullability
+        """
+        plain_type, element_is_optional = self._resolve_union(avro_type[inner_field_name])
+        if isinstance(plain_type, dict):
+            # We need the element-id downstream
+            plain_type["field-id"] = avro_type[id_field]
+        inner_field = self._convert_schema(plain_type)
+        return inner_field, element_is_optional
+
+    def _convert_schema(self, avro_type: str | Dict[str, Any]) -> IcebergType:
+        """
+        Resolves the Avro type
+
+        Args:
+            avro_type: The Avro type, can be simple or complex
+
+        Returns:
+            The equivalent IcebergType
+
+        Raises:
+            ValueError: When there are unknown types
+        """
+        if isinstance(avro_type, str):
+            return PRIMITIVE_FIELD_TYPE_MAPPING[avro_type]
+        elif isinstance(avro_type, dict):
+            if "logicalType" in avro_type:
+                return self._convert_logical_type(avro_type)
+            else:
+                type_identifier = avro_type["type"]
+                if type_identifier == "record":
+                    return self._convert_record_type(avro_type)
+                elif type_identifier == "array":
+                    return self._convert_array_type(avro_type)
+                elif type_identifier == "map":
+                    return self._convert_map_type(avro_type)
+                elif type_identifier == "fixed":
+                    return self._convert_fixed_type(avro_type)
+                else:
+                    raise ValueError(f"Unknown type: {avro_type}")
+        else:
+            raise ValueError(f"Unknown type: {avro_type}")
+
+    def _convert_field(self, field: Dict[str, Any]) -> NestedField:
+        """
+        Converts an Avro field into an Iceberg equivalent field
+        Args:
+            field: The Avro field
+
+        Returns:
+            The Iceberg equivalent field
+        """
+        assert "field-id" in field, "Missing field-id in the Avro field, this is required for converting it to an Iceberg schema"

Review Comment:
   What is thrown when using `assert` rather than `raise ValueError`?
   
   Also, we try to be more direct with error messages by starting with what was attempted, then giving context for what went wrong. It's also assumed that if an exception is raised then some requirement wasn't met so we'd typically leave out the "this is required ..." part. How about `f"Cannot convert field, missing field-id: {field}"`?



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
Fokko commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r877583312


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,274 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Util class for converting between Avro and Iceberg schemas
+
+"""
+import logging
+from typing import Any, Dict, List, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DoubleType,
+    FloatType,
+    IntegerType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+)
+
+logger = logging.getLogger(__name__)
+
+
+class AvroSchemaConversion:
+    PRIMITIVE_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "boolean": BooleanType(),
+        "bytes": BinaryType(),
+        "date": DateType(),
+        "double": DoubleType(),
+        "float": FloatType(),
+        "int": IntegerType(),
+        "long": LongType(),
+        "string": StringType(),
+        "time-millis": TimeType(),
+        "timestamp-millis": TimestampType(),

Review Comment:
   Added with 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] rdblue commented on a diff in pull request #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r883233050


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,371 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Utility class for converting between Avro and Iceberg schemas
+
+"""
+import logging
+from typing import Any, Dict, List, Optional, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IcebergType,
+    IntegerType,
+    ListType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+    UUIDType,
+)
+
+logger = logging.getLogger(__name__)
+
+
+class AvroSchemaConversion:
+    PRIMITIVE_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "boolean": BooleanType(),
+        "bytes": BinaryType(),
+        "double": DoubleType(),
+        "float": FloatType(),
+        "int": IntegerType(),
+        "long": LongType(),
+        "string": StringType(),
+    }
+
+    LOGICAL_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "date": DateType(),
+        "time-millis": TimeType(),
+        "timestamp-millis": TimestampType(),
+        "uuid": UUIDType(),
+    }
+
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expects to have field id's to be encoded in the Avro schema::

Review Comment:
   You may want to mention in the docs that this handles string types, like "long".



-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
Fokko commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r883871218


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,371 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Utility class for converting between Avro and Iceberg schemas
+
+"""
+import logging
+from typing import Any, Dict, List, Optional, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IcebergType,
+    IntegerType,
+    ListType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+    UUIDType,
+)
+
+logger = logging.getLogger(__name__)
+
+
+class AvroSchemaConversion:
+    PRIMITIVE_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "boolean": BooleanType(),
+        "bytes": BinaryType(),
+        "double": DoubleType(),
+        "float": FloatType(),
+        "int": IntegerType(),
+        "long": LongType(),
+        "string": StringType(),
+    }
+
+    LOGICAL_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "date": DateType(),
+        "time-millis": TimeType(),
+        "timestamp-millis": TimestampType(),
+        "uuid": UUIDType(),
+    }
+
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expects to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into a Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+        """
+        fields = self._parse_record(avro_schema)
+        return Schema(*fields.fields, schema_id=1)
+
+    def _parse_record(self, avro_field: Dict[str, Any]) -> StructType:
+        fields = [self._parse_field(field) for field in avro_field["fields"]]
+        return StructType(*fields)  # type: ignore
+
+    def _resolve_union(self, type_union: Union[Dict, List, str]) -> Tuple[Union[str, Dict[str, Any]], bool]:

Review Comment:
   Yes, I'll bump this to the touched files. 
   
   Also, there is a nice tool called pyupgrade that will enforce this: https://github.com/asottile/pyupgrade#pep-604-typing-rewrites I'll do this in a separate PR, and probably after https://github.com/apache/iceberg/pull/4811



-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
Fokko commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r884323061


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,451 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Utility class for converting between Avro and Iceberg schemas
+
+"""
+from __future__ import annotations
+
+import logging
+from typing import Any, Dict, List, Tuple
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IcebergType,
+    IntegerType,
+    ListType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+    UUIDType,
+)
+
+logger = logging.getLogger(__name__)
+
+PRIMITIVE_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "boolean": BooleanType(),
+    "bytes": BinaryType(),
+    "double": DoubleType(),
+    "float": FloatType(),
+    "int": IntegerType(),
+    "long": LongType(),
+    "string": StringType(),
+}
+
+LOGICAL_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "date": DateType(),
+    "time-millis": TimeType(),
+    "timestamp-millis": TimestampType(),
+    "uuid": UUIDType(),
+}
+
+
+class AvroSchemaConversion:
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expects to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into an Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+        """
+        return Schema(*[self._convert_field(field) for field in avro_schema["fields"]], schema_id=1)
+
+    def _resolve_union(self, type_union: Dict | List | str) -> Tuple[str | Dict[str, Any], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        Raises:
+            TypeError: In the case non-optional union types are encountered
+        """
+        avro_types: Dict | List
+        if isinstance(type_union, str):
+            # It is a primitive and required
+            return type_union, False
+        elif isinstance(type_union, dict):
+            # It is a context and required
+            return type_union, False
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        if len(avro_types) > 2:
+            raise TypeError("Non-optional types aren't part of the Iceberg specification")
+
+        # Filter the null value and return the type
+        return list(filter(lambda t: t != "null", avro_types))[0], is_optional
+
+    def _resolve_inner_type(self, avro_type: Dict[str, Any], inner_field_name: str, id_field: str) -> Tuple[IcebergType, bool]:
+        """
+        Removes any unions inside an array or map, and appends the field-id downstream (if required)
+
+        Args:
+            avro_type: The parent Avro type
+            inner_field_name: For example 'items' in case of an array
+            id_field: Contains the element or key/value id, for example element-id, to be copied to the field-id
+
+        Returns:
+            The inner type and the nullability
+        """
+        plain_type, element_is_optional = self._resolve_union(avro_type[inner_field_name])
+        if isinstance(plain_type, dict):
+            # We need the element-id downstream
+            plain_type["field-id"] = avro_type[id_field]
+        inner_field = self._convert_schema(plain_type)
+        return inner_field, element_is_optional
+
+    def _convert_schema(self, avro_type: str | Dict[str, Any]) -> IcebergType:
+        """
+        Resolves the Avro type
+
+        Args:
+            avro_type: The Avro type, can be simple or complex
+
+        Returns:
+            The equivalent IcebergType
+
+        Raises:
+            ValueError: When there are unknown types
+        """
+        if isinstance(avro_type, str):
+            return PRIMITIVE_FIELD_TYPE_MAPPING[avro_type]
+        elif isinstance(avro_type, dict):
+            if "logicalType" in avro_type:
+                return self._convert_logical_type(avro_type)
+            else:
+                type_identifier = avro_type["type"]
+                if type_identifier == "record":
+                    return self._convert_record_type(avro_type)
+                elif type_identifier == "array":
+                    return self._convert_array_type(avro_type)
+                elif type_identifier == "map":
+                    return self._convert_map_type(avro_type)
+                elif type_identifier == "fixed":
+                    return self._convert_fixed_type(avro_type)
+                else:
+                    raise ValueError(f"Unknown type: {avro_type}")
+        else:
+            raise ValueError(f"Unknown type: {avro_type}")
+
+    def _convert_field(self, field: Dict[str, Any]) -> NestedField:
+        """
+        Converts an Avro field into an Iceberg equivalent field
+        Args:
+            field: The Avro field
+
+        Returns:
+            The Iceberg equivalent field
+        """
+        assert "field-id" in field, "Missing field-id in the Avro field, this is required for converting it to an Iceberg schema"
+
+        plain_type, is_optional = self._resolve_union(field["type"])
+
+        return NestedField(
+            field_id=field["field-id"],
+            name=field["name"],
+            field_type=self._convert_schema(plain_type),
+            is_optional=is_optional,
+            doc=field.get("doc"),
+        )
+
+    def _convert_record_type(self, record_type: Dict[str, Any]) -> StructType:
+        """
+        Converts the fields from a record into an Iceberg struct
+
+        Examples:
+            >>> from iceberg.utils.schema_conversion import AvroSchemaConversion
+            >>> record_type = {
+            ...     "type": "record",
+            ...     "name": "r508",
+            ...     "fields": [{
+            ...         "name": "contains_null",
+            ...         "type": "boolean",
+            ...         "doc": "True if any file has a null partition value",
+            ...         "field-id": 509,
+            ...      }, {
+            ...          "name": "contains_nan",
+            ...          "type": ["null", "boolean"],
+            ...          "doc": "True if any file has a nan partition value",
+            ...          "default": None,
+            ...          "field-id": 518,
+            ...      }],
+            ... }
+            >>> actual = AvroSchemaConversion()._convert_record_type(record_type)
+            >>> expected = StructType(
+            ...     fields=(
+            ...         NestedField(
+            ...             field_id=509,
+            ...             name="contains_null",
+            ...             field_type=BooleanType(),
+            ...             is_optional=False,
+            ...             doc="True if any file has a null partition value",
+            ...         ),
+            ...         NestedField(
+            ...             field_id=518,
+            ...             name="contains_nan",
+            ...             field_type=BooleanType(),
+            ...             is_optional=True,
+            ...             doc="True if any file has a nan partition value",
+            ...         ),
+            ...     )
+            ... )
+            >>> expected == actual
+            True
+
+        Args:
+            record_type: The record type itself
+
+        Returns:
+        """
+        return StructType(*[self._convert_field(field) for field in record_type["fields"]])
+
+    def _convert_array_type(self, array_type: Dict[str, Any]) -> ListType:
+        element_type, element_is_optional = self._resolve_inner_type(array_type, "items", "element-id")
+        return ListType(element_id=array_type["element-id"], element_type=element_type, element_is_optional=element_is_optional)
+
+    def _convert_map_type(self, map_type: Dict[str, Any]) -> MapType:
+        """
+        Args:
+            map_type: The dict that describes the Avro map type
+
+        Examples:
+            >>> from iceberg.utils.schema_conversion import AvroSchemaConversion
+            >>> avro_field = {
+            ...     "type": "map",
+            ...     "values": ["long", "null"],
+            ...     "key-id": 101,
+            ...     "value-id": 102,
+            ... }
+            >>> actual = AvroSchemaConversion()._convert_map_type(avro_field)
+            >>> expected = MapType(
+            ...     key_id=101,
+            ...     key_type=StringType(),
+            ...     value_id=102,
+            ...     value_type=LongType(),
+            ...     value_is_optional=True
+            ... )
+            >>> actual == expected
+            True
+
+        Returns: A MapType
+        """
+        value_type, value_is_optional = self._resolve_union(map_type["values"])
+        return MapType(
+            key_id=map_type["key-id"],
+            # Avro only supports string keys
+            key_type=StringType(),
+            value_id=map_type["value-id"],
+            value_type=self._convert_schema(value_type),
+            value_is_optional=value_is_optional,
+        )
+
+    def _convert_logical_type(self, avro_logical_type: Dict[str, Any]) -> IcebergType:
+        """
+        When a logical type is found, we'll resolve it here. For the decimal and map
+        we need to fetch more keys from the dict, and for the simple ones we can just
+        look it up in the mapping.
+
+        Examples:
+            >>> from iceberg.utils.schema_conversion import AvroSchemaConversion
+            >>> avro_logical_type = {
+            ...     "type": "int",
+            ...     "logicalType": "date"
+            ... }
+            >>> actual = AvroSchemaConversion()._convert_logical_type(avro_logical_type)
+            >>> actual == DateType()
+            True
+
+        Args:
+            avro_logical_type: The logical type
+
+        Returns:
+            The converted logical type
+
+        Raises:
+            ValueError: When the logical type is unknown
+        """
+        logical_type = avro_logical_type["logicalType"]
+        if logical_type == "decimal":
+            return self._convert_logical_decimal_type(avro_logical_type)
+        elif logical_type == "map":
+            return self._convert_logical_map_type(avro_logical_type)
+        elif logical_type in LOGICAL_FIELD_TYPE_MAPPING:
+            return LOGICAL_FIELD_TYPE_MAPPING[logical_type]
+        else:
+            raise ValueError(f"Unknown logical type: {avro_logical_type}")
+
+    def _convert_logical_decimal_type(self, avro_type: Dict[str, Any]) -> DecimalType:
+        """
+        Args:
+            avro_type: The Avro type
+
+        Examples:
+            >>> from iceberg.utils.schema_conversion import AvroSchemaConversion
+            >>> avro_decimal_type = {
+            ...     "type": "bytes",
+            ...     "logicalType": "decimal",
+            ...     "precision": 19,
+            ...     "scale": 25
+            ... }
+            >>> actual = AvroSchemaConversion()._convert_logical_decimal_type(avro_decimal_type)
+            >>> expected = DecimalType(
+            ...     precision=19,
+            ...     scale=25
+            ... )
+            >>> actual == expected
+            True
+
+        Returns:
+            A Iceberg DecimalType
+        """
+        return DecimalType(precision=avro_type["precision"], scale=avro_type["scale"])
+
+    def _convert_logical_map_type(self, avro_type: Dict[str, Any]) -> MapType:
+        """
+        In the case where a map hasn't a key as a type you can use a logical map to
+        still encode this in Avro
+
+        Args:
+            avro_type: The Avro Type
+
+        Examples:
+            >>> from iceberg.utils.schema_conversion import AvroSchemaConversion
+            >>> avro_type = {
+            ...     "type": "array",
+            ...     "logicalType": "map",
+            ...     "items": {
+            ...         "type": "record",
+            ...         "name": "k101_v102",
+            ...         "fields": [
+            ...             {"name": "key", "type": "int", "field-id": 101},
+            ...             {"name": "value", "type": "string", "field-id": 102},
+            ...         ],
+            ...     },
+            ... }
+            >>> actual = AvroSchemaConversion()._convert_logical_map_type(avro_type)
+            >>> expected = MapType(
+            ...         key_id=101,
+            ...         key_type=IntegerType(),
+            ...         value_id=102,
+            ...         value_type=StringType(),
+            ...         value_is_optional=False
+            ... )
+            >>> actual == expected
+            True
+
+        .. _Apache Iceberg specification:
+            https://iceberg.apache.org/spec/#appendix-a-format-specific-requirements
+
+        Returns:
+            The logical map
+        """
+        fields = avro_type["items"]["fields"]
+        assert len(fields) == 2, f"Expected two fields in the logical map, but got: {fields}"
+        key = self._convert_field(fields[0])
+        value = self._convert_field(fields[1])

Review Comment:
   Replaced it by:
   ```python
   key = self._convert_field(list(filter(lambda f: f["name"] == "key", fields))[0])
   value = self._convert_field(list(filter(lambda f: f["name"] == "value", fields))[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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r884326410


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,454 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Utility class for converting between Avro and Iceberg schemas
+
+"""
+from __future__ import annotations
+
+import logging
+from typing import Any, Dict, List, Tuple
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IcebergType,
+    IntegerType,
+    ListType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+    UUIDType,
+)
+
+logger = logging.getLogger(__name__)
+
+PRIMITIVE_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "boolean": BooleanType(),
+    "bytes": BinaryType(),
+    "double": DoubleType(),
+    "float": FloatType(),
+    "int": IntegerType(),
+    "long": LongType(),
+    "string": StringType(),
+    "enum": StringType(),
+}
+
+LOGICAL_FIELD_TYPE_MAPPING: Dict[Tuple[str, str], PrimitiveType] = {
+    ("date", "int"): DateType(),
+    ("time-millis", "int"): TimeType(),
+    ("timestamp-millis", "long"): TimestampType(),
+    ("time-micros", "int"): TimeType(),
+    ("timestamp-micros", "long"): TimestampType(),
+    ("uuid", "string"): UUIDType(),
+}
+
+
+class AvroSchemaConversion:
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expects to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into an Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+        """
+        return Schema(*[self._convert_field(field) for field in avro_schema["fields"]], schema_id=1)
+
+    def _resolve_union(self, type_union: Dict | List | str) -> Tuple[str | Dict[str, Any], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        Raises:
+            TypeError: In the case non-optional union types are encountered
+        """
+        avro_types: Dict | List
+        if isinstance(type_union, str):
+            # It is a primitive and required
+            return type_union, False
+        elif isinstance(type_union, dict):
+            # It is a context and required
+            return type_union, False
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        if len(avro_types) > 2:
+            raise TypeError("Non-optional types aren't part of the Iceberg specification")
+
+        # Filter the null value and return the type
+        return list(filter(lambda t: t != "null", avro_types))[0], is_optional
+
+    def _convert_schema(self, avro_type: str | Dict[str, Any]) -> IcebergType:
+        """
+        Resolves the Avro type
+
+        Args:
+            avro_type: The Avro type, can be simple or complex
+
+        Returns:
+            The equivalent IcebergType
+
+        Raises:
+            ValueError: When there are unknown types
+        """
+        if isinstance(avro_type, str):
+            return PRIMITIVE_FIELD_TYPE_MAPPING[avro_type]
+        elif isinstance(avro_type, dict):
+            if "logicalType" in avro_type:
+                return self._convert_logical_type(avro_type)
+            else:
+                # Resolve potential nested types
+                while "type" in avro_type and isinstance(avro_type["type"], dict):
+                    avro_type = avro_type["type"]
+                type_identifier = avro_type["type"]
+                if type_identifier == "record":
+                    return self._convert_record_type(avro_type)
+                elif type_identifier == "array":
+                    return self._convert_array_type(avro_type)
+                elif type_identifier == "map":
+                    return self._convert_map_type(avro_type)
+                elif type_identifier == "fixed":
+                    return self._convert_fixed_type(avro_type)
+                elif isinstance(type_identifier, str):
+                    return PRIMITIVE_FIELD_TYPE_MAPPING[type_identifier]
+                else:
+                    raise ValueError(f"Unknown type: {avro_type}")
+        else:
+            raise ValueError(f"Unknown type: {avro_type}")
+
+    def _convert_field(self, field: Dict[str, Any]) -> NestedField:
+        """
+        Converts an Avro field into an Iceberg equivalent field
+        Args:
+            field: The Avro field
+
+        Returns:
+            The Iceberg equivalent field
+        """
+        if "field-id" not in field:
+            raise ValueError(f"Cannot convert field, missing field-id: {field}")
+
+        plain_type, is_optional = self._resolve_union(field["type"])
+
+        return NestedField(
+            field_id=field["field-id"],
+            name=field["name"],
+            field_type=self._convert_schema(plain_type),
+            is_optional=is_optional,
+            doc=field.get("doc"),
+        )
+
+    def _convert_record_type(self, record_type: Dict[str, Any]) -> StructType:
+        """
+        Converts the fields from a record into an Iceberg struct
+
+        Examples:
+            >>> from iceberg.utils.schema_conversion import AvroSchemaConversion
+            >>> record_type = {
+            ...     "type": "record",
+            ...     "name": "r508",
+            ...     "fields": [{
+            ...         "name": "contains_null",
+            ...         "type": "boolean",
+            ...         "doc": "True if any file has a null partition value",
+            ...         "field-id": 509,
+            ...      }, {
+            ...          "name": "contains_nan",
+            ...          "type": ["null", "boolean"],
+            ...          "doc": "True if any file has a nan partition value",
+            ...          "default": None,
+            ...          "field-id": 518,
+            ...      }],
+            ... }
+            >>> actual = AvroSchemaConversion()._convert_record_type(record_type)
+            >>> expected = StructType(
+            ...     fields=(
+            ...         NestedField(
+            ...             field_id=509,
+            ...             name="contains_null",
+            ...             field_type=BooleanType(),
+            ...             is_optional=False,
+            ...             doc="True if any file has a null partition value",
+            ...         ),
+            ...         NestedField(
+            ...             field_id=518,
+            ...             name="contains_nan",
+            ...             field_type=BooleanType(),
+            ...             is_optional=True,
+            ...             doc="True if any file has a nan partition value",
+            ...         ),
+            ...     )
+            ... )
+            >>> expected == actual
+            True
+
+        Args:
+            record_type: The record type itself
+
+        Returns:
+        """
+        if record_type["type"] != "record":
+            raise ValueError(f"Expected type, got: {record_type}")
+
+        return StructType(*[self._convert_field(field) for field in record_type["fields"]])
+
+    def _convert_array_type(self, array_type: Dict[str, Any]) -> ListType:
+        if "element-id" not in array_type:
+            raise ValueError(f"Cannot convert array-type, missing element-id: {array_type}")
+
+        plain_type, element_is_optional = self._resolve_union(array_type["items"])
+
+        return ListType(
+            element_id=array_type["element-id"],
+            element_type=self._convert_schema(plain_type),
+            element_is_optional=element_is_optional,
+        )
+
+    def _convert_map_type(self, map_type: Dict[str, Any]) -> MapType:
+        """
+        Args:
+            map_type: The dict that describes the Avro map type
+
+        Examples:
+            >>> from iceberg.utils.schema_conversion import AvroSchemaConversion
+            >>> avro_field = {
+            ...     "type": "map",
+            ...     "values": ["long", "null"],
+            ...     "key-id": 101,
+            ...     "value-id": 102,
+            ... }
+            >>> actual = AvroSchemaConversion()._convert_map_type(avro_field)
+            >>> expected = MapType(
+            ...     key_id=101,
+            ...     key_type=StringType(),
+            ...     value_id=102,
+            ...     value_type=LongType(),
+            ...     value_is_optional=True
+            ... )
+            >>> actual == expected
+            True
+
+        Returns: A MapType
+        """
+        value_type, value_is_optional = self._resolve_union(map_type["values"])
+        return MapType(
+            key_id=map_type["key-id"],
+            # Avro only supports string keys
+            key_type=StringType(),
+            value_id=map_type["value-id"],
+            value_type=self._convert_schema(value_type),
+            value_is_optional=value_is_optional,
+        )
+
+    def _convert_logical_type(self, avro_logical_type: Dict[str, Any]) -> IcebergType:
+        """
+        Convert a schema with a logical type annotation. For the decimal and map
+        we need to fetch more keys from the dict, and for the simple ones we can just
+        look it up in the mapping.
+
+        Examples:
+            >>> from iceberg.utils.schema_conversion import AvroSchemaConversion
+            >>> avro_logical_type = {
+            ...     "type": "int",
+            ...     "logicalType": "date"
+            ... }
+            >>> actual = AvroSchemaConversion()._convert_logical_type(avro_logical_type)
+            >>> actual == DateType()
+            True
+
+        Args:
+            avro_logical_type: The logical type
+
+        Returns:
+            The converted logical type
+
+        Raises:
+            ValueError: When the logical type is unknown
+        """
+        logical_type = avro_logical_type["logicalType"]
+        physical_type = avro_logical_type["type"]
+        if logical_type == "decimal":
+            return self._convert_logical_decimal_type(avro_logical_type)
+        elif logical_type == "map":
+            return self._convert_logical_map_type(avro_logical_type)
+        elif (logical_type, physical_type) in LOGICAL_FIELD_TYPE_MAPPING:
+            return LOGICAL_FIELD_TYPE_MAPPING[(logical_type, physical_type)]

Review Comment:
   I just also realized that this doesn't check for the `adjustToUtc` flag that we use to distinguish between `TimestampType()` and `TimestamptzType()`. We'll have to add that.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
Fokko commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r876241608


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,274 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Util class for converting between Avro and Iceberg schemas
+
+"""
+import logging
+from typing import Any, Dict, List, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DoubleType,
+    FloatType,
+    IntegerType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+)
+
+logger = logging.getLogger(__name__)
+
+
+class AvroSchemaConversion:
+    PRIMITIVE_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "boolean": BooleanType(),
+        "bytes": BinaryType(),
+        "date": DateType(),
+        "double": DoubleType(),
+        "float": FloatType(),
+        "int": IntegerType(),
+        "long": LongType(),
+        "string": StringType(),
+        "time-millis": TimeType(),
+        "timestamp-millis": TimestampType(),
+    }
+
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expect to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into a Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+
+        Todo:
+            * Implement full support for unions
+            * Implement logical types
+        """
+        fields = self._parse_record(avro_schema)
+        return Schema(*fields.fields, schema_id=1)  # type: ignore
+
+    def _parse_record(self, avro_field: Dict[str, Any]) -> StructType:
+        return StructType(*[self._parse_field(field) for field in avro_field["fields"]])
+
+    def _resolve_union(self, type_union: Union[Dict, List, str]) -> Tuple[Union[str, dict], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        """
+        avro_types: Union[Dict, List]
+        if isinstance(type_union, str):
+            avro_types = [type_union]
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        # Filter the null value, so we know the actual type
+        avro_types = list(filter(lambda t: t != "null", avro_types))
+
+        if len(avro_types) != 1:
+            raise ValueError("Support for unions is yet to be implemented")

Review Comment:
   Good to know, I'll update the error! 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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
Fokko commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r877582558


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,274 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Util class for converting between Avro and Iceberg schemas
+
+"""
+import logging
+from typing import Any, Dict, List, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DoubleType,
+    FloatType,
+    IntegerType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+)
+
+logger = logging.getLogger(__name__)
+
+
+class AvroSchemaConversion:
+    PRIMITIVE_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "boolean": BooleanType(),
+        "bytes": BinaryType(),
+        "date": DateType(),
+        "double": DoubleType(),
+        "float": FloatType(),
+        "int": IntegerType(),
+        "long": LongType(),
+        "string": StringType(),
+        "time-millis": TimeType(),
+        "timestamp-millis": TimestampType(),

Review Comment:
   Done! Added a test for the date, and split the logical types out on their own map.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
Fokko commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r877591158


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,274 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Util class for converting between Avro and Iceberg schemas
+
+"""
+import logging
+from typing import Any, Dict, List, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DoubleType,
+    FloatType,
+    IntegerType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+)
+
+logger = logging.getLogger(__name__)
+
+
+class AvroSchemaConversion:
+    PRIMITIVE_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "boolean": BooleanType(),
+        "bytes": BinaryType(),
+        "date": DateType(),
+        "double": DoubleType(),
+        "float": FloatType(),
+        "int": IntegerType(),
+        "long": LongType(),
+        "string": StringType(),
+        "time-millis": TimeType(),
+        "timestamp-millis": TimestampType(),
+    }
+
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expect to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into a Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+
+        Todo:
+            * Implement full support for unions
+            * Implement logical types
+        """
+        fields = self._parse_record(avro_schema)
+        return Schema(*fields.fields, schema_id=1)  # type: ignore
+
+    def _parse_record(self, avro_field: Dict[str, Any]) -> StructType:
+        return StructType(*[self._parse_field(field) for field in avro_field["fields"]])
+
+    def _resolve_union(self, type_union: Union[Dict, List, str]) -> Tuple[Union[str, dict], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        """
+        avro_types: Union[Dict, List]
+        if isinstance(type_union, str):
+            avro_types = [type_union]
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        # Filter the null value, so we know the actual type
+        avro_types = list(filter(lambda t: t != "null", avro_types))
+
+        if len(avro_types) != 1:
+            raise ValueError("Support for unions is yet to be implemented")
+
+        avro_type = avro_types[0]
+
+        return avro_type, is_optional
+
+    def _parse_field(self, field: Dict[str, Any]) -> NestedField:
+        """
+        Recursively walks through the Schema, constructing the Iceberg schema
+
+        Examples:
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {
+            ...             "name": "partitions",
+            ...                "type": [
+            ...                "null",
+            ...                {
+            ...                        "type": "array",
+            ...                        "items": {
+            ...                            "type": "record",
+            ...                            "name": "r508",
+            ...                            "fields": [
+            ...                                {
+            ...                                    "name": "contains_null",
+            ...                                    "type": "boolean",
+            ...                                    "doc": "True if any file has a null partition value",
+            ...                                    "field-id": 509,
+            ...                                },
+            ...                                {
+            ...                                    "name": "contains_nan",
+            ...                                    "type": ["null", "boolean"],
+            ...                                    "doc": "True if any file has a nan partition value",
+            ...                                    "default": None,
+            ...                                    "field-id": 518,
+            ...                                },
+            ...                            ],
+            ...                        },
+            ...                        "element-id": 508,
+            ...                    },
+            ...                ],
+            ...                "doc": "Summary for each partition",
+            ...                "default": None,
+            ...                "field-id": 507,
+            ...            },
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=507,
+            ...         name="partitions",
+            ...         field_type=StructType(
+            ...             NestedField(
+            ...                 field_id=509,
+            ...                 name="contains_null",
+            ...                 field_type=BooleanType(),
+            ...                 is_optional=False,
+            ...                 doc="True if any file has a null partition value",
+            ...             ),
+            ...             NestedField(
+            ...                 field_id=518,
+            ...                 name="contains_nan",
+            ...                 field_type=BooleanType(),
+            ...                 is_optional=True,
+            ...                 doc="True if any file has a nan partition value",
+            ...             ),
+            ...         ),
+            ...         is_optional=True,
+            ...         doc="Summary for each partition",
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            field:
+
+        Returns:
+
+        """
+        field_id = field["field-id"]
+        field_name = field["name"]
+        field_doc = field.get("doc")
+
+        avro_type, is_optional = self._resolve_union(field["type"])
+        if isinstance(avro_type, dict):
+            avro_type = avro_type["type"]
+
+        if isinstance(avro_type, str) and avro_type in AvroSchemaConversion.PRIMITIVE_FIELD_TYPE_MAP:
+            return NestedField(
+                field_id=field_id,
+                name=field_name,
+                field_type=AvroSchemaConversion.PRIMITIVE_FIELD_TYPE_MAP[avro_type],
+                is_optional=is_optional,
+                doc=field_doc,
+            )
+        elif avro_type == "record":
+            return NestedField(
+                field_id=field_id, name=field_name, field_type=self._parse_record(field), is_optional=is_optional, doc=field_doc
+            )
+        elif avro_type == "array":

Review Comment:
   Great catch, thanks! Just added it with a test



-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#issuecomment-1132271452

   Thanks! I'll take another look.


-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r884298382


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,451 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Utility class for converting between Avro and Iceberg schemas
+
+"""
+from __future__ import annotations
+
+import logging
+from typing import Any, Dict, List, Tuple
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IcebergType,
+    IntegerType,
+    ListType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+    UUIDType,
+)
+
+logger = logging.getLogger(__name__)
+
+PRIMITIVE_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "boolean": BooleanType(),
+    "bytes": BinaryType(),
+    "double": DoubleType(),
+    "float": FloatType(),
+    "int": IntegerType(),
+    "long": LongType(),
+    "string": StringType(),
+}
+
+LOGICAL_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "date": DateType(),
+    "time-millis": TimeType(),
+    "timestamp-millis": TimestampType(),
+    "uuid": UUIDType(),
+}
+
+
+class AvroSchemaConversion:
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expects to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into an Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+        """
+        return Schema(*[self._convert_field(field) for field in avro_schema["fields"]], schema_id=1)
+
+    def _resolve_union(self, type_union: Dict | List | str) -> Tuple[str | Dict[str, Any], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        Raises:
+            TypeError: In the case non-optional union types are encountered
+        """
+        avro_types: Dict | List
+        if isinstance(type_union, str):
+            # It is a primitive and required
+            return type_union, False
+        elif isinstance(type_union, dict):
+            # It is a context and required
+            return type_union, False
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        if len(avro_types) > 2:
+            raise TypeError("Non-optional types aren't part of the Iceberg specification")
+
+        # Filter the null value and return the type
+        return list(filter(lambda t: t != "null", avro_types))[0], is_optional
+
+    def _resolve_inner_type(self, avro_type: Dict[str, Any], inner_field_name: str, id_field: str) -> Tuple[IcebergType, bool]:
+        """
+        Removes any unions inside an array or map, and appends the field-id downstream (if required)
+
+        Args:
+            avro_type: The parent Avro type
+            inner_field_name: For example 'items' in case of an array
+            id_field: Contains the element or key/value id, for example element-id, to be copied to the field-id
+
+        Returns:
+            The inner type and the nullability
+        """
+        plain_type, element_is_optional = self._resolve_union(avro_type[inner_field_name])
+        if isinstance(plain_type, dict):
+            # We need the element-id downstream
+            plain_type["field-id"] = avro_type[id_field]
+        inner_field = self._convert_schema(plain_type)
+        return inner_field, element_is_optional
+
+    def _convert_schema(self, avro_type: str | Dict[str, Any]) -> IcebergType:
+        """
+        Resolves the Avro type
+
+        Args:
+            avro_type: The Avro type, can be simple or complex
+
+        Returns:
+            The equivalent IcebergType
+
+        Raises:
+            ValueError: When there are unknown types
+        """
+        if isinstance(avro_type, str):
+            return PRIMITIVE_FIELD_TYPE_MAPPING[avro_type]
+        elif isinstance(avro_type, dict):
+            if "logicalType" in avro_type:
+                return self._convert_logical_type(avro_type)
+            else:
+                type_identifier = avro_type["type"]
+                if type_identifier == "record":
+                    return self._convert_record_type(avro_type)
+                elif type_identifier == "array":
+                    return self._convert_array_type(avro_type)
+                elif type_identifier == "map":
+                    return self._convert_map_type(avro_type)
+                elif type_identifier == "fixed":
+                    return self._convert_fixed_type(avro_type)
+                else:
+                    raise ValueError(f"Unknown type: {avro_type}")
+        else:
+            raise ValueError(f"Unknown type: {avro_type}")
+
+    def _convert_field(self, field: Dict[str, Any]) -> NestedField:
+        """
+        Converts an Avro field into an Iceberg equivalent field
+        Args:
+            field: The Avro field
+
+        Returns:
+            The Iceberg equivalent field
+        """
+        assert "field-id" in field, "Missing field-id in the Avro field, this is required for converting it to an Iceberg schema"
+
+        plain_type, is_optional = self._resolve_union(field["type"])
+
+        return NestedField(
+            field_id=field["field-id"],
+            name=field["name"],
+            field_type=self._convert_schema(plain_type),
+            is_optional=is_optional,
+            doc=field.get("doc"),
+        )
+
+    def _convert_record_type(self, record_type: Dict[str, Any]) -> StructType:
+        """
+        Converts the fields from a record into an Iceberg struct
+
+        Examples:
+            >>> from iceberg.utils.schema_conversion import AvroSchemaConversion
+            >>> record_type = {
+            ...     "type": "record",
+            ...     "name": "r508",
+            ...     "fields": [{
+            ...         "name": "contains_null",
+            ...         "type": "boolean",
+            ...         "doc": "True if any file has a null partition value",
+            ...         "field-id": 509,
+            ...      }, {
+            ...          "name": "contains_nan",
+            ...          "type": ["null", "boolean"],
+            ...          "doc": "True if any file has a nan partition value",
+            ...          "default": None,
+            ...          "field-id": 518,
+            ...      }],
+            ... }
+            >>> actual = AvroSchemaConversion()._convert_record_type(record_type)
+            >>> expected = StructType(
+            ...     fields=(
+            ...         NestedField(
+            ...             field_id=509,
+            ...             name="contains_null",
+            ...             field_type=BooleanType(),
+            ...             is_optional=False,
+            ...             doc="True if any file has a null partition value",
+            ...         ),
+            ...         NestedField(
+            ...             field_id=518,
+            ...             name="contains_nan",
+            ...             field_type=BooleanType(),
+            ...             is_optional=True,
+            ...             doc="True if any file has a nan partition value",
+            ...         ),
+            ...     )
+            ... )
+            >>> expected == actual
+            True
+
+        Args:
+            record_type: The record type itself
+
+        Returns:
+        """
+        return StructType(*[self._convert_field(field) for field in record_type["fields"]])
+
+    def _convert_array_type(self, array_type: Dict[str, Any]) -> ListType:
+        element_type, element_is_optional = self._resolve_inner_type(array_type, "items", "element-id")
+        return ListType(element_id=array_type["element-id"], element_type=element_type, element_is_optional=element_is_optional)
+
+    def _convert_map_type(self, map_type: Dict[str, Any]) -> MapType:
+        """
+        Args:
+            map_type: The dict that describes the Avro map type
+
+        Examples:
+            >>> from iceberg.utils.schema_conversion import AvroSchemaConversion
+            >>> avro_field = {
+            ...     "type": "map",
+            ...     "values": ["long", "null"],
+            ...     "key-id": 101,
+            ...     "value-id": 102,
+            ... }
+            >>> actual = AvroSchemaConversion()._convert_map_type(avro_field)
+            >>> expected = MapType(
+            ...     key_id=101,
+            ...     key_type=StringType(),
+            ...     value_id=102,
+            ...     value_type=LongType(),
+            ...     value_is_optional=True
+            ... )
+            >>> actual == expected
+            True
+
+        Returns: A MapType
+        """
+        value_type, value_is_optional = self._resolve_union(map_type["values"])
+        return MapType(
+            key_id=map_type["key-id"],
+            # Avro only supports string keys
+            key_type=StringType(),
+            value_id=map_type["value-id"],
+            value_type=self._convert_schema(value_type),
+            value_is_optional=value_is_optional,
+        )
+
+    def _convert_logical_type(self, avro_logical_type: Dict[str, Any]) -> IcebergType:
+        """
+        When a logical type is found, we'll resolve it here. For the decimal and map
+        we need to fetch more keys from the dict, and for the simple ones we can just
+        look it up in the mapping.
+
+        Examples:
+            >>> from iceberg.utils.schema_conversion import AvroSchemaConversion
+            >>> avro_logical_type = {
+            ...     "type": "int",
+            ...     "logicalType": "date"
+            ... }
+            >>> actual = AvroSchemaConversion()._convert_logical_type(avro_logical_type)
+            >>> actual == DateType()
+            True
+
+        Args:
+            avro_logical_type: The logical type
+
+        Returns:
+            The converted logical type
+
+        Raises:
+            ValueError: When the logical type is unknown
+        """
+        logical_type = avro_logical_type["logicalType"]
+        if logical_type == "decimal":
+            return self._convert_logical_decimal_type(avro_logical_type)
+        elif logical_type == "map":
+            return self._convert_logical_map_type(avro_logical_type)
+        elif logical_type in LOGICAL_FIELD_TYPE_MAPPING:
+            return LOGICAL_FIELD_TYPE_MAPPING[logical_type]
+        else:
+            raise ValueError(f"Unknown logical type: {avro_logical_type}")
+
+    def _convert_logical_decimal_type(self, avro_type: Dict[str, Any]) -> DecimalType:
+        """
+        Args:
+            avro_type: The Avro type
+
+        Examples:
+            >>> from iceberg.utils.schema_conversion import AvroSchemaConversion
+            >>> avro_decimal_type = {
+            ...     "type": "bytes",
+            ...     "logicalType": "decimal",
+            ...     "precision": 19,
+            ...     "scale": 25
+            ... }
+            >>> actual = AvroSchemaConversion()._convert_logical_decimal_type(avro_decimal_type)
+            >>> expected = DecimalType(
+            ...     precision=19,
+            ...     scale=25
+            ... )
+            >>> actual == expected
+            True
+
+        Returns:
+            A Iceberg DecimalType
+        """
+        return DecimalType(precision=avro_type["precision"], scale=avro_type["scale"])
+
+    def _convert_logical_map_type(self, avro_type: Dict[str, Any]) -> MapType:
+        """
+        In the case where a map hasn't a key as a type you can use a logical map to
+        still encode this in Avro
+
+        Args:
+            avro_type: The Avro Type
+
+        Examples:
+            >>> from iceberg.utils.schema_conversion import AvroSchemaConversion
+            >>> avro_type = {
+            ...     "type": "array",
+            ...     "logicalType": "map",
+            ...     "items": {
+            ...         "type": "record",
+            ...         "name": "k101_v102",
+            ...         "fields": [
+            ...             {"name": "key", "type": "int", "field-id": 101},
+            ...             {"name": "value", "type": "string", "field-id": 102},
+            ...         ],
+            ...     },
+            ... }
+            >>> actual = AvroSchemaConversion()._convert_logical_map_type(avro_type)
+            >>> expected = MapType(
+            ...         key_id=101,
+            ...         key_type=IntegerType(),
+            ...         value_id=102,
+            ...         value_type=StringType(),
+            ...         value_is_optional=False
+            ... )
+            >>> actual == expected
+            True
+
+        .. _Apache Iceberg specification:
+            https://iceberg.apache.org/spec/#appendix-a-format-specific-requirements
+
+        Returns:
+            The logical map
+        """
+        fields = avro_type["items"]["fields"]
+        assert len(fields) == 2, f"Expected two fields in the logical map, but got: {fields}"

Review Comment:
   How about `Invalid key-value pair schema: {avro_type["items"]}`?



-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#issuecomment-1140528137

   There are a few minor things to fix, but overall I think this looks good. I'm going to commit it to unblock reader work and we can follow up with minor updates. Thanks, @Fokko!


-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r883234500


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,371 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Utility class for converting between Avro and Iceberg schemas
+
+"""
+import logging
+from typing import Any, Dict, List, Optional, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IcebergType,
+    IntegerType,
+    ListType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+    UUIDType,
+)
+
+logger = logging.getLogger(__name__)
+
+
+class AvroSchemaConversion:
+    PRIMITIVE_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "boolean": BooleanType(),
+        "bytes": BinaryType(),
+        "double": DoubleType(),
+        "float": FloatType(),
+        "int": IntegerType(),
+        "long": LongType(),
+        "string": StringType(),
+    }
+
+    LOGICAL_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "date": DateType(),
+        "time-millis": TimeType(),
+        "timestamp-millis": TimestampType(),
+        "uuid": UUIDType(),
+    }
+
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expects to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into a Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+        """
+        fields = self._parse_record(avro_schema)
+        return Schema(*fields.fields, schema_id=1)
+
+    def _parse_record(self, avro_field: Dict[str, Any]) -> StructType:

Review Comment:
   Nit: `avro_field` is a misleading name because it's actually a record.



-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
Fokko commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r883871218


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,371 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Utility class for converting between Avro and Iceberg schemas
+
+"""
+import logging
+from typing import Any, Dict, List, Optional, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IcebergType,
+    IntegerType,
+    ListType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+    UUIDType,
+)
+
+logger = logging.getLogger(__name__)
+
+
+class AvroSchemaConversion:
+    PRIMITIVE_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "boolean": BooleanType(),
+        "bytes": BinaryType(),
+        "double": DoubleType(),
+        "float": FloatType(),
+        "int": IntegerType(),
+        "long": LongType(),
+        "string": StringType(),
+    }
+
+    LOGICAL_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "date": DateType(),
+        "time-millis": TimeType(),
+        "timestamp-millis": TimestampType(),
+        "uuid": UUIDType(),
+    }
+
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expects to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into a Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+        """
+        fields = self._parse_record(avro_schema)
+        return Schema(*fields.fields, schema_id=1)
+
+    def _parse_record(self, avro_field: Dict[str, Any]) -> StructType:
+        fields = [self._parse_field(field) for field in avro_field["fields"]]
+        return StructType(*fields)  # type: ignore
+
+    def _resolve_union(self, type_union: Union[Dict, List, str]) -> Tuple[Union[str, Dict[str, Any]], bool]:

Review Comment:
   Yes, let me bulk update this in another PR. There is a nice tool called pyupgrade that will enforce this: https://github.com/asottile/pyupgrade#pep-604-typing-rewrites



-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
Fokko commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r883865974


##########
python/src/iceberg/types.py:
##########
@@ -146,7 +146,7 @@ class NestedField(IcebergType):
     name: str = field()
     field_type: IcebergType = field()
     is_optional: bool = field(default=True)
-    doc: Optional[str] = field(default=None, repr=False)
+    doc: Optional[str] = field(default=None, repr=False, compare=False, hash=False)

Review Comment:
   Let's include it here for now. I'll revert this change and update the 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] rdblue commented on a diff in pull request #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r884027554


##########
python/src/iceberg/schema.py:
##########
@@ -67,6 +67,17 @@ def __repr__(self):
             f"Schema(fields={repr(self.columns)}, schema_id={self.schema_id}, identifier_field_ids={self.identifier_field_ids})"
         )
 
+    def __eq__(self, other) -> bool:

Review Comment:
   In that case, I think we'll need to check the identifier field IDs like Kyle suggested



-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r884296326


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,451 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Utility class for converting between Avro and Iceberg schemas
+
+"""
+from __future__ import annotations
+
+import logging
+from typing import Any, Dict, List, Tuple
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IcebergType,
+    IntegerType,
+    ListType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+    UUIDType,
+)
+
+logger = logging.getLogger(__name__)
+
+PRIMITIVE_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "boolean": BooleanType(),
+    "bytes": BinaryType(),
+    "double": DoubleType(),
+    "float": FloatType(),
+    "int": IntegerType(),
+    "long": LongType(),
+    "string": StringType(),
+}
+
+LOGICAL_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "date": DateType(),
+    "time-millis": TimeType(),
+    "timestamp-millis": TimestampType(),
+    "uuid": UUIDType(),
+}
+
+
+class AvroSchemaConversion:
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expects to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into an Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+        """
+        return Schema(*[self._convert_field(field) for field in avro_schema["fields"]], schema_id=1)
+
+    def _resolve_union(self, type_union: Dict | List | str) -> Tuple[str | Dict[str, Any], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        Raises:
+            TypeError: In the case non-optional union types are encountered
+        """
+        avro_types: Dict | List
+        if isinstance(type_union, str):
+            # It is a primitive and required
+            return type_union, False
+        elif isinstance(type_union, dict):
+            # It is a context and required
+            return type_union, False
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        if len(avro_types) > 2:
+            raise TypeError("Non-optional types aren't part of the Iceberg specification")
+
+        # Filter the null value and return the type
+        return list(filter(lambda t: t != "null", avro_types))[0], is_optional
+
+    def _resolve_inner_type(self, avro_type: Dict[str, Any], inner_field_name: str, id_field: str) -> Tuple[IcebergType, bool]:

Review Comment:
   Nit: since this is used by array and map conversion, it would make more sense to have it near those methods.



-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r883880962


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,371 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Utility class for converting between Avro and Iceberg schemas
+
+"""
+import logging
+from typing import Any, Dict, List, Optional, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IcebergType,
+    IntegerType,
+    ListType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+    UUIDType,
+)
+
+logger = logging.getLogger(__name__)
+
+
+class AvroSchemaConversion:
+    PRIMITIVE_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "boolean": BooleanType(),
+        "bytes": BinaryType(),
+        "double": DoubleType(),
+        "float": FloatType(),
+        "int": IntegerType(),
+        "long": LongType(),
+        "string": StringType(),
+    }
+
+    LOGICAL_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "date": DateType(),
+        "time-millis": TimeType(),
+        "timestamp-millis": TimestampType(),
+        "uuid": UUIDType(),
+    }
+
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expects to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into a Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+        """
+        fields = self._parse_record(avro_schema)
+        return Schema(*fields.fields, schema_id=1)
+
+    def _parse_record(self, avro_field: Dict[str, Any]) -> StructType:
+        fields = [self._parse_field(field) for field in avro_field["fields"]]
+        return StructType(*fields)  # type: ignore
+
+    def _resolve_union(self, type_union: Union[Dict, List, str]) -> Tuple[Union[str, Dict[str, Any]], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        Raises:
+            TypeError: In the case non-optional union types are encountered
+        """
+        avro_types: Union[Dict, List]
+        if isinstance(type_union, str):
+            # It is a primitive and required
+            return type_union, False
+        elif isinstance(type_union, dict):
+            # It is a context and required
+            return type_union, False
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types

Review Comment:
   The check for number of non-null branches would catch that. But `['string', 'bytes']` does not contain null, which is what you want 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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
Fokko commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r884314966


##########
python/src/iceberg/schema.py:
##########
@@ -62,8 +62,23 @@ def __repr__(self):
             f"Schema(fields={repr(self.columns)}, schema_id={self.schema_id}, identifier_field_ids={self.identifier_field_ids})"
         )
 
+    def __eq__(self, other) -> bool:
+        if not other:
+            return False
+
+        if not isinstance(other, Schema):
+            return False
+
+        if len(self.columns) != len(other.columns):
+            return False
+
+        identifier_field_ids_is_equal = self.identifier_field_ids == other.identifier_field_ids
+        schema_is_equal = all([lhs == rhs for lhs, rhs in zip(self.columns, other.columns)])
+
+        return identifier_field_ids_is_equal and schema_is_equal
+
     @property
-    def columns(self) -> Iterable[NestedField]:
+    def columns(self) -> Tuple[NestedField]:

Review Comment:
   Yes, good one. I know that we're going a bit back and forth on this annotation, but I've double-checked using the inspector, and it is a Tuple at runtime 👍🏻 



-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r872838503


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,274 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Util class for converting between Avro and Iceberg schemas
+
+"""
+import logging
+from typing import Any, Dict, List, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DoubleType,
+    FloatType,
+    IntegerType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+)
+
+logger = logging.getLogger(__name__)
+
+
+class AvroSchemaConversion:
+    PRIMITIVE_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "boolean": BooleanType(),
+        "bytes": BinaryType(),
+        "date": DateType(),
+        "double": DoubleType(),
+        "float": FloatType(),
+        "int": IntegerType(),
+        "long": LongType(),
+        "string": StringType(),
+        "time-millis": TimeType(),
+        "timestamp-millis": TimestampType(),
+    }
+
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expect to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into a Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+
+        Todo:
+            * Implement full support for unions
+            * Implement logical types
+        """
+        fields = self._parse_record(avro_schema)
+        return Schema(*fields.fields, schema_id=1)  # type: ignore
+
+    def _parse_record(self, avro_field: Dict[str, Any]) -> StructType:
+        return StructType(*[self._parse_field(field) for field in avro_field["fields"]])
+
+    def _resolve_union(self, type_union: Union[Dict, List, str]) -> Tuple[Union[str, dict], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        """
+        avro_types: Union[Dict, List]
+        if isinstance(type_union, str):
+            avro_types = [type_union]
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        # Filter the null value, so we know the actual type
+        avro_types = list(filter(lambda t: t != "null", avro_types))
+
+        if len(avro_types) != 1:
+            raise ValueError("Support for unions is yet to be implemented")
+
+        avro_type = avro_types[0]
+
+        return avro_type, is_optional
+
+    def _parse_field(self, field: Dict[str, Any]) -> NestedField:
+        """
+        Recursively walks through the Schema, constructing the Iceberg schema
+
+        Examples:
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {
+            ...             "name": "partitions",
+            ...                "type": [
+            ...                "null",
+            ...                {
+            ...                        "type": "array",
+            ...                        "items": {
+            ...                            "type": "record",
+            ...                            "name": "r508",
+            ...                            "fields": [
+            ...                                {
+            ...                                    "name": "contains_null",
+            ...                                    "type": "boolean",
+            ...                                    "doc": "True if any file has a null partition value",
+            ...                                    "field-id": 509,
+            ...                                },
+            ...                                {
+            ...                                    "name": "contains_nan",
+            ...                                    "type": ["null", "boolean"],
+            ...                                    "doc": "True if any file has a nan partition value",
+            ...                                    "default": None,
+            ...                                    "field-id": 518,
+            ...                                },
+            ...                            ],
+            ...                        },
+            ...                        "element-id": 508,
+            ...                    },
+            ...                ],
+            ...                "doc": "Summary for each partition",
+            ...                "default": None,
+            ...                "field-id": 507,
+            ...            },
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=507,
+            ...         name="partitions",
+            ...         field_type=StructType(
+            ...             NestedField(
+            ...                 field_id=509,
+            ...                 name="contains_null",
+            ...                 field_type=BooleanType(),
+            ...                 is_optional=False,
+            ...                 doc="True if any file has a null partition value",
+            ...             ),
+            ...             NestedField(
+            ...                 field_id=518,
+            ...                 name="contains_nan",
+            ...                 field_type=BooleanType(),
+            ...                 is_optional=True,
+            ...                 doc="True if any file has a nan partition value",
+            ...             ),
+            ...         ),
+            ...         is_optional=True,
+            ...         doc="Summary for each partition",
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            field:
+
+        Returns:
+
+        """
+        field_id = field["field-id"]
+        field_name = field["name"]
+        field_doc = field.get("doc")
+
+        avro_type, is_optional = self._resolve_union(field["type"])
+        if isinstance(avro_type, dict):
+            avro_type = avro_type["type"]
+
+        if isinstance(avro_type, str) and avro_type in AvroSchemaConversion.PRIMITIVE_FIELD_TYPE_MAP:
+            return NestedField(
+                field_id=field_id,
+                name=field_name,
+                field_type=AvroSchemaConversion.PRIMITIVE_FIELD_TYPE_MAP[avro_type],
+                is_optional=is_optional,
+                doc=field_doc,
+            )
+        elif avro_type == "record":
+            return NestedField(
+                field_id=field_id, name=field_name, field_type=self._parse_record(field), is_optional=is_optional, doc=field_doc
+            )
+        elif avro_type == "array":
+            inner_type, _ = self._resolve_union(field["type"])
+            assert isinstance(inner_type, dict)

Review Comment:
   If this fails, do we get a good error message? I generally prefer to throw a `ValueError` with a good message if there's a chance anyone would hit 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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r872638896


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,274 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Util class for converting between Avro and Iceberg schemas
+
+"""
+import logging
+from typing import Any, Dict, List, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DoubleType,
+    FloatType,
+    IntegerType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+)
+
+logger = logging.getLogger(__name__)
+
+
+class AvroSchemaConversion:
+    PRIMITIVE_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "boolean": BooleanType(),
+        "bytes": BinaryType(),
+        "date": DateType(),
+        "double": DoubleType(),
+        "float": FloatType(),
+        "int": IntegerType(),
+        "long": LongType(),
+        "string": StringType(),
+        "time-millis": TimeType(),
+        "timestamp-millis": TimestampType(),
+    }
+
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expect to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into a Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+
+        Todo:
+            * Implement full support for unions
+            * Implement logical types
+        """
+        fields = self._parse_record(avro_schema)
+        return Schema(*fields.fields, schema_id=1)  # type: ignore
+
+    def _parse_record(self, avro_field: Dict[str, Any]) -> StructType:
+        return StructType(*[self._parse_field(field) for field in avro_field["fields"]])
+
+    def _resolve_union(self, type_union: Union[Dict, List, str]) -> Tuple[Union[str, dict], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        """
+        avro_types: Union[Dict, List]
+        if isinstance(type_union, str):
+            avro_types = [type_union]
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        # Filter the null value, so we know the actual type
+        avro_types = list(filter(lambda t: t != "null", avro_types))
+
+        if len(avro_types) != 1:
+            raise ValueError("Support for unions is yet to be implemented")

Review Comment:
   This is outside the Iceberg spec (although #4242 is proposing a way to handle it in Java).
   
   I think this can just state that the type is invalid because it is a non-option union.



-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r872839779


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,274 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Util class for converting between Avro and Iceberg schemas
+
+"""
+import logging
+from typing import Any, Dict, List, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DoubleType,
+    FloatType,
+    IntegerType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+)
+
+logger = logging.getLogger(__name__)
+
+
+class AvroSchemaConversion:
+    PRIMITIVE_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "boolean": BooleanType(),
+        "bytes": BinaryType(),
+        "date": DateType(),
+        "double": DoubleType(),
+        "float": FloatType(),
+        "int": IntegerType(),
+        "long": LongType(),
+        "string": StringType(),
+        "time-millis": TimeType(),
+        "timestamp-millis": TimestampType(),
+    }
+
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expect to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into a Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+
+        Todo:
+            * Implement full support for unions
+            * Implement logical types
+        """
+        fields = self._parse_record(avro_schema)
+        return Schema(*fields.fields, schema_id=1)  # type: ignore
+
+    def _parse_record(self, avro_field: Dict[str, Any]) -> StructType:
+        return StructType(*[self._parse_field(field) for field in avro_field["fields"]])
+
+    def _resolve_union(self, type_union: Union[Dict, List, str]) -> Tuple[Union[str, dict], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        """
+        avro_types: Union[Dict, List]
+        if isinstance(type_union, str):
+            avro_types = [type_union]
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        # Filter the null value, so we know the actual type
+        avro_types = list(filter(lambda t: t != "null", avro_types))
+
+        if len(avro_types) != 1:
+            raise ValueError("Support for unions is yet to be implemented")
+
+        avro_type = avro_types[0]
+
+        return avro_type, is_optional
+
+    def _parse_field(self, field: Dict[str, Any]) -> NestedField:
+        """
+        Recursively walks through the Schema, constructing the Iceberg schema
+
+        Examples:
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {
+            ...             "name": "partitions",
+            ...                "type": [
+            ...                "null",
+            ...                {
+            ...                        "type": "array",
+            ...                        "items": {
+            ...                            "type": "record",
+            ...                            "name": "r508",
+            ...                            "fields": [
+            ...                                {
+            ...                                    "name": "contains_null",
+            ...                                    "type": "boolean",
+            ...                                    "doc": "True if any file has a null partition value",
+            ...                                    "field-id": 509,
+            ...                                },
+            ...                                {
+            ...                                    "name": "contains_nan",
+            ...                                    "type": ["null", "boolean"],
+            ...                                    "doc": "True if any file has a nan partition value",
+            ...                                    "default": None,
+            ...                                    "field-id": 518,
+            ...                                },
+            ...                            ],
+            ...                        },
+            ...                        "element-id": 508,
+            ...                    },
+            ...                ],
+            ...                "doc": "Summary for each partition",
+            ...                "default": None,
+            ...                "field-id": 507,
+            ...            },
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=507,
+            ...         name="partitions",
+            ...         field_type=StructType(
+            ...             NestedField(
+            ...                 field_id=509,
+            ...                 name="contains_null",
+            ...                 field_type=BooleanType(),
+            ...                 is_optional=False,
+            ...                 doc="True if any file has a null partition value",
+            ...             ),
+            ...             NestedField(
+            ...                 field_id=518,
+            ...                 name="contains_nan",
+            ...                 field_type=BooleanType(),
+            ...                 is_optional=True,
+            ...                 doc="True if any file has a nan partition value",
+            ...             ),
+            ...         ),
+            ...         is_optional=True,
+            ...         doc="Summary for each partition",
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            field:
+
+        Returns:
+
+        """
+        field_id = field["field-id"]
+        field_name = field["name"]
+        field_doc = field.get("doc")
+
+        avro_type, is_optional = self._resolve_union(field["type"])
+        if isinstance(avro_type, dict):
+            avro_type = avro_type["type"]
+
+        if isinstance(avro_type, str) and avro_type in AvroSchemaConversion.PRIMITIVE_FIELD_TYPE_MAP:
+            return NestedField(
+                field_id=field_id,
+                name=field_name,
+                field_type=AvroSchemaConversion.PRIMITIVE_FIELD_TYPE_MAP[avro_type],
+                is_optional=is_optional,
+                doc=field_doc,
+            )
+        elif avro_type == "record":
+            return NestedField(
+                field_id=field_id, name=field_name, field_type=self._parse_record(field), is_optional=is_optional, doc=field_doc
+            )
+        elif avro_type == "array":
+            inner_type, _ = self._resolve_union(field["type"])
+            assert isinstance(inner_type, dict)
+            inner_type["items"]["field-id"] = inner_type["element-id"]

Review Comment:
   I don't think there is a guarantee that `"items"` returns a `dict`. The example from the Avro spec is `{ "type": "array", "items": "string", "default": [] }` so we know `"items"` can be a 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] rdblue commented on a diff in pull request #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r872842831


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,274 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Util class for converting between Avro and Iceberg schemas
+
+"""
+import logging
+from typing import Any, Dict, List, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DoubleType,
+    FloatType,
+    IntegerType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+)
+
+logger = logging.getLogger(__name__)
+
+
+class AvroSchemaConversion:
+    PRIMITIVE_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "boolean": BooleanType(),
+        "bytes": BinaryType(),
+        "date": DateType(),
+        "double": DoubleType(),
+        "float": FloatType(),
+        "int": IntegerType(),
+        "long": LongType(),
+        "string": StringType(),
+        "time-millis": TimeType(),
+        "timestamp-millis": TimestampType(),
+    }
+
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expect to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into a Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+
+        Todo:
+            * Implement full support for unions
+            * Implement logical types
+        """
+        fields = self._parse_record(avro_schema)
+        return Schema(*fields.fields, schema_id=1)  # type: ignore
+
+    def _parse_record(self, avro_field: Dict[str, Any]) -> StructType:
+        return StructType(*[self._parse_field(field) for field in avro_field["fields"]])
+
+    def _resolve_union(self, type_union: Union[Dict, List, str]) -> Tuple[Union[str, dict], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        """
+        avro_types: Union[Dict, List]
+        if isinstance(type_union, str):
+            avro_types = [type_union]
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        # Filter the null value, so we know the actual type
+        avro_types = list(filter(lambda t: t != "null", avro_types))
+
+        if len(avro_types) != 1:
+            raise ValueError("Support for unions is yet to be implemented")
+
+        avro_type = avro_types[0]
+
+        return avro_type, is_optional
+
+    def _parse_field(self, field: Dict[str, Any]) -> NestedField:
+        """
+        Recursively walks through the Schema, constructing the Iceberg schema
+
+        Examples:
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {
+            ...             "name": "partitions",
+            ...                "type": [
+            ...                "null",
+            ...                {
+            ...                        "type": "array",
+            ...                        "items": {
+            ...                            "type": "record",
+            ...                            "name": "r508",
+            ...                            "fields": [
+            ...                                {
+            ...                                    "name": "contains_null",
+            ...                                    "type": "boolean",
+            ...                                    "doc": "True if any file has a null partition value",
+            ...                                    "field-id": 509,
+            ...                                },
+            ...                                {
+            ...                                    "name": "contains_nan",
+            ...                                    "type": ["null", "boolean"],
+            ...                                    "doc": "True if any file has a nan partition value",
+            ...                                    "default": None,
+            ...                                    "field-id": 518,
+            ...                                },
+            ...                            ],
+            ...                        },
+            ...                        "element-id": 508,
+            ...                    },
+            ...                ],
+            ...                "doc": "Summary for each partition",
+            ...                "default": None,
+            ...                "field-id": 507,
+            ...            },
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=507,
+            ...         name="partitions",
+            ...         field_type=StructType(
+            ...             NestedField(
+            ...                 field_id=509,
+            ...                 name="contains_null",
+            ...                 field_type=BooleanType(),
+            ...                 is_optional=False,
+            ...                 doc="True if any file has a null partition value",
+            ...             ),
+            ...             NestedField(
+            ...                 field_id=518,
+            ...                 name="contains_nan",
+            ...                 field_type=BooleanType(),
+            ...                 is_optional=True,
+            ...                 doc="True if any file has a nan partition value",
+            ...             ),
+            ...         ),
+            ...         is_optional=True,
+            ...         doc="Summary for each partition",
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            field:
+
+        Returns:
+
+        """
+        field_id = field["field-id"]
+        field_name = field["name"]
+        field_doc = field.get("doc")
+
+        avro_type, is_optional = self._resolve_union(field["type"])
+        if isinstance(avro_type, dict):
+            avro_type = avro_type["type"]
+
+        if isinstance(avro_type, str) and avro_type in AvroSchemaConversion.PRIMITIVE_FIELD_TYPE_MAP:
+            return NestedField(
+                field_id=field_id,
+                name=field_name,
+                field_type=AvroSchemaConversion.PRIMITIVE_FIELD_TYPE_MAP[avro_type],
+                is_optional=is_optional,
+                doc=field_doc,
+            )
+        elif avro_type == "record":
+            return NestedField(
+                field_id=field_id, name=field_name, field_type=self._parse_record(field), is_optional=is_optional, doc=field_doc
+            )
+        elif avro_type == "array":

Review Comment:
   I think you'll have to check for a logical type annotation here that signals a map with a non-string key.



-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
Fokko commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r877583159


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,274 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Util class for converting between Avro and Iceberg schemas
+
+"""
+import logging
+from typing import Any, Dict, List, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DoubleType,
+    FloatType,
+    IntegerType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+)
+
+logger = logging.getLogger(__name__)
+
+
+class AvroSchemaConversion:
+    PRIMITIVE_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "boolean": BooleanType(),
+        "bytes": BinaryType(),
+        "date": DateType(),
+        "double": DoubleType(),
+        "float": FloatType(),
+        "int": IntegerType(),
+        "long": LongType(),
+        "string": StringType(),
+        "time-millis": TimeType(),
+        "timestamp-millis": TimestampType(),
+    }
+
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expect to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into a Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+
+        Todo:
+            * Implement full support for unions
+            * Implement logical types
+        """
+        fields = self._parse_record(avro_schema)
+        return Schema(*fields.fields, schema_id=1)  # type: ignore
+
+    def _parse_record(self, avro_field: Dict[str, Any]) -> StructType:
+        return StructType(*[self._parse_field(field) for field in avro_field["fields"]])
+
+    def _resolve_union(self, type_union: Union[Dict, List, str]) -> Tuple[Union[str, dict], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        """
+        avro_types: Union[Dict, List]
+        if isinstance(type_union, str):
+            avro_types = [type_union]
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        # Filter the null value, so we know the actual type
+        avro_types = list(filter(lambda t: t != "null", avro_types))
+
+        if len(avro_types) != 1:
+            raise ValueError("Support for unions is yet to be implemented")
+
+        avro_type = avro_types[0]
+
+        return avro_type, is_optional
+
+    def _parse_field(self, field: Dict[str, Any]) -> NestedField:
+        """
+        Recursively walks through the Schema, constructing the Iceberg schema
+
+        Examples:
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {
+            ...             "name": "partitions",
+            ...                "type": [
+            ...                "null",
+            ...                {
+            ...                        "type": "array",
+            ...                        "items": {
+            ...                            "type": "record",
+            ...                            "name": "r508",
+            ...                            "fields": [
+            ...                                {
+            ...                                    "name": "contains_null",
+            ...                                    "type": "boolean",
+            ...                                    "doc": "True if any file has a null partition value",
+            ...                                    "field-id": 509,
+            ...                                },
+            ...                                {
+            ...                                    "name": "contains_nan",
+            ...                                    "type": ["null", "boolean"],
+            ...                                    "doc": "True if any file has a nan partition value",
+            ...                                    "default": None,
+            ...                                    "field-id": 518,
+            ...                                },
+            ...                            ],
+            ...                        },
+            ...                        "element-id": 508,
+            ...                    },
+            ...                ],
+            ...                "doc": "Summary for each partition",
+            ...                "default": None,
+            ...                "field-id": 507,
+            ...            },
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=507,
+            ...         name="partitions",
+            ...         field_type=StructType(
+            ...             NestedField(
+            ...                 field_id=509,
+            ...                 name="contains_null",
+            ...                 field_type=BooleanType(),
+            ...                 is_optional=False,
+            ...                 doc="True if any file has a null partition value",
+            ...             ),
+            ...             NestedField(
+            ...                 field_id=518,
+            ...                 name="contains_nan",
+            ...                 field_type=BooleanType(),
+            ...                 is_optional=True,
+            ...                 doc="True if any file has a nan partition value",
+            ...             ),
+            ...         ),
+            ...         is_optional=True,
+            ...         doc="Summary for each partition",
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            field:
+
+        Returns:
+
+        """
+        field_id = field["field-id"]
+        field_name = field["name"]
+        field_doc = field.get("doc")
+
+        avro_type, is_optional = self._resolve_union(field["type"])
+        if isinstance(avro_type, dict):
+            avro_type = avro_type["type"]
+
+        if isinstance(avro_type, str) and avro_type in AvroSchemaConversion.PRIMITIVE_FIELD_TYPE_MAP:
+            return NestedField(
+                field_id=field_id,
+                name=field_name,
+                field_type=AvroSchemaConversion.PRIMITIVE_FIELD_TYPE_MAP[avro_type],
+                is_optional=is_optional,
+                doc=field_doc,
+            )
+        elif avro_type == "record":
+            return NestedField(
+                field_id=field_id, name=field_name, field_type=self._parse_record(field), is_optional=is_optional, doc=field_doc
+            )
+        elif avro_type == "array":
+            inner_type, _ = self._resolve_union(field["type"])
+            assert isinstance(inner_type, dict)

Review Comment:
   I've refactored this and added tests for the permutations:
   - `"str"`
   - `["str", "null"]`
   - `["null", { record }]`



-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
Fokko commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r876242694


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,274 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Util class for converting between Avro and Iceberg schemas
+
+"""
+import logging
+from typing import Any, Dict, List, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DoubleType,
+    FloatType,
+    IntegerType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+)
+
+logger = logging.getLogger(__name__)
+
+
+class AvroSchemaConversion:
+    PRIMITIVE_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "boolean": BooleanType(),
+        "bytes": BinaryType(),
+        "date": DateType(),
+        "double": DoubleType(),
+        "float": FloatType(),
+        "int": IntegerType(),
+        "long": LongType(),
+        "string": StringType(),
+        "time-millis": TimeType(),
+        "timestamp-millis": TimestampType(),
+    }
+
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expect to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into a Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+
+        Todo:
+            * Implement full support for unions
+            * Implement logical types
+        """
+        fields = self._parse_record(avro_schema)
+        return Schema(*fields.fields, schema_id=1)  # type: ignore
+
+    def _parse_record(self, avro_field: Dict[str, Any]) -> StructType:
+        return StructType(*[self._parse_field(field) for field in avro_field["fields"]])
+
+    def _resolve_union(self, type_union: Union[Dict, List, str]) -> Tuple[Union[str, dict], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        """
+        avro_types: Union[Dict, List]
+        if isinstance(type_union, str):
+            avro_types = [type_union]

Review Comment:
   Good one, I'll update it



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r883231803


##########
python/src/iceberg/schema.py:
##########
@@ -67,6 +67,17 @@ def __repr__(self):
             f"Schema(fields={repr(self.columns)}, schema_id={self.schema_id}, identifier_field_ids={self.identifier_field_ids})"
         )
 
+    def __eq__(self, other) -> bool:

Review Comment:
   Does this `__eq__` implementation override the autogenerated one? If we're using a generated one, I don't understand why this would still be 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] Fokko commented on a diff in pull request #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
Fokko commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r883865027


##########
python/src/iceberg/schema.py:
##########
@@ -67,6 +67,17 @@ def __repr__(self):
             f"Schema(fields={repr(self.columns)}, schema_id={self.schema_id}, identifier_field_ids={self.identifier_field_ids})"
         )
 
+    def __eq__(self, other) -> bool:

Review Comment:
   Yes, that's true. I forgot to remove this one, 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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
Fokko commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r883979735


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,371 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Utility class for converting between Avro and Iceberg schemas
+
+"""
+import logging
+from typing import Any, Dict, List, Optional, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IcebergType,
+    IntegerType,
+    ListType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+    UUIDType,
+)
+
+logger = logging.getLogger(__name__)
+
+
+class AvroSchemaConversion:
+    PRIMITIVE_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "boolean": BooleanType(),
+        "bytes": BinaryType(),
+        "double": DoubleType(),
+        "float": FloatType(),
+        "int": IntegerType(),
+        "long": LongType(),
+        "string": StringType(),
+    }
+
+    LOGICAL_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "date": DateType(),
+        "time-millis": TimeType(),
+        "timestamp-millis": TimestampType(),
+        "uuid": UUIDType(),
+    }
+
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expects to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into a Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+        """
+        fields = self._parse_record(avro_schema)
+        return Schema(*fields.fields, schema_id=1)
+
+    def _parse_record(self, avro_field: Dict[str, Any]) -> StructType:
+        fields = [self._parse_field(field) for field in avro_field["fields"]]
+        return StructType(*fields)  # type: ignore
+
+    def _resolve_union(self, type_union: Union[Dict, List, str]) -> Tuple[Union[str, Dict[str, Any]], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        Raises:
+            TypeError: In the case non-optional union types are encountered
+        """
+        avro_types: Union[Dict, List]
+        if isinstance(type_union, str):
+            # It is a primitive and required
+            return type_union, False
+        elif isinstance(type_union, dict):
+            # It is a context and required
+            return type_union, False
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types

Review Comment:
   You're right! I've updated 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


[GitHub] [iceberg] Fokko commented on a diff in pull request #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
Fokko commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r884321655


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,451 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Utility class for converting between Avro and Iceberg schemas
+
+"""
+from __future__ import annotations
+
+import logging
+from typing import Any, Dict, List, Tuple
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IcebergType,
+    IntegerType,
+    ListType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+    UUIDType,
+)
+
+logger = logging.getLogger(__name__)
+
+PRIMITIVE_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "boolean": BooleanType(),
+    "bytes": BinaryType(),
+    "double": DoubleType(),
+    "float": FloatType(),
+    "int": IntegerType(),
+    "long": LongType(),
+    "string": StringType(),
+}
+
+LOGICAL_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "date": DateType(),
+    "time-millis": TimeType(),
+    "timestamp-millis": TimestampType(),
+    "uuid": UUIDType(),
+}
+
+
+class AvroSchemaConversion:
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expects to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into an Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+        """
+        return Schema(*[self._convert_field(field) for field in avro_schema["fields"]], schema_id=1)
+
+    def _resolve_union(self, type_union: Dict | List | str) -> Tuple[str | Dict[str, Any], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        Raises:
+            TypeError: In the case non-optional union types are encountered
+        """
+        avro_types: Dict | List
+        if isinstance(type_union, str):
+            # It is a primitive and required
+            return type_union, False
+        elif isinstance(type_union, dict):
+            # It is a context and required
+            return type_union, False
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        if len(avro_types) > 2:
+            raise TypeError("Non-optional types aren't part of the Iceberg specification")
+
+        # Filter the null value and return the type
+        return list(filter(lambda t: t != "null", avro_types))[0], is_optional
+
+    def _resolve_inner_type(self, avro_type: Dict[str, Any], inner_field_name: str, id_field: str) -> Tuple[IcebergType, bool]:
+        """
+        Removes any unions inside an array or map, and appends the field-id downstream (if required)
+
+        Args:
+            avro_type: The parent Avro type
+            inner_field_name: For example 'items' in case of an array
+            id_field: Contains the element or key/value id, for example element-id, to be copied to the field-id
+
+        Returns:
+            The inner type and the nullability
+        """
+        plain_type, element_is_optional = self._resolve_union(avro_type[inner_field_name])
+        if isinstance(plain_type, dict):
+            # We need the element-id downstream
+            plain_type["field-id"] = avro_type[id_field]
+        inner_field = self._convert_schema(plain_type)
+        return inner_field, element_is_optional
+
+    def _convert_schema(self, avro_type: str | Dict[str, Any]) -> IcebergType:
+        """
+        Resolves the Avro type
+
+        Args:
+            avro_type: The Avro type, can be simple or complex
+
+        Returns:
+            The equivalent IcebergType
+
+        Raises:
+            ValueError: When there are unknown types
+        """
+        if isinstance(avro_type, str):
+            return PRIMITIVE_FIELD_TYPE_MAPPING[avro_type]
+        elif isinstance(avro_type, dict):
+            if "logicalType" in avro_type:
+                return self._convert_logical_type(avro_type)
+            else:
+                type_identifier = avro_type["type"]
+                if type_identifier == "record":
+                    return self._convert_record_type(avro_type)
+                elif type_identifier == "array":
+                    return self._convert_array_type(avro_type)
+                elif type_identifier == "map":
+                    return self._convert_map_type(avro_type)
+                elif type_identifier == "fixed":
+                    return self._convert_fixed_type(avro_type)
+                else:
+                    raise ValueError(f"Unknown type: {avro_type}")

Review Comment:
   Good one, haven't seen that myself. Added a test and logic for handling 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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r884297732


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,451 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Utility class for converting between Avro and Iceberg schemas
+
+"""
+from __future__ import annotations
+
+import logging
+from typing import Any, Dict, List, Tuple
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IcebergType,
+    IntegerType,
+    ListType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+    UUIDType,
+)
+
+logger = logging.getLogger(__name__)
+
+PRIMITIVE_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "boolean": BooleanType(),
+    "bytes": BinaryType(),
+    "double": DoubleType(),
+    "float": FloatType(),
+    "int": IntegerType(),
+    "long": LongType(),
+    "string": StringType(),
+}
+
+LOGICAL_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "date": DateType(),
+    "time-millis": TimeType(),
+    "timestamp-millis": TimestampType(),
+    "uuid": UUIDType(),
+}
+
+
+class AvroSchemaConversion:
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expects to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into an Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+        """
+        return Schema(*[self._convert_field(field) for field in avro_schema["fields"]], schema_id=1)
+
+    def _resolve_union(self, type_union: Dict | List | str) -> Tuple[str | Dict[str, Any], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        Raises:
+            TypeError: In the case non-optional union types are encountered
+        """
+        avro_types: Dict | List
+        if isinstance(type_union, str):
+            # It is a primitive and required
+            return type_union, False
+        elif isinstance(type_union, dict):
+            # It is a context and required
+            return type_union, False
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        if len(avro_types) > 2:
+            raise TypeError("Non-optional types aren't part of the Iceberg specification")
+
+        # Filter the null value and return the type
+        return list(filter(lambda t: t != "null", avro_types))[0], is_optional
+
+    def _resolve_inner_type(self, avro_type: Dict[str, Any], inner_field_name: str, id_field: str) -> Tuple[IcebergType, bool]:
+        """
+        Removes any unions inside an array or map, and appends the field-id downstream (if required)
+
+        Args:
+            avro_type: The parent Avro type
+            inner_field_name: For example 'items' in case of an array
+            id_field: Contains the element or key/value id, for example element-id, to be copied to the field-id
+
+        Returns:
+            The inner type and the nullability
+        """
+        plain_type, element_is_optional = self._resolve_union(avro_type[inner_field_name])
+        if isinstance(plain_type, dict):
+            # We need the element-id downstream
+            plain_type["field-id"] = avro_type[id_field]
+        inner_field = self._convert_schema(plain_type)
+        return inner_field, element_is_optional
+
+    def _convert_schema(self, avro_type: str | Dict[str, Any]) -> IcebergType:
+        """
+        Resolves the Avro type
+
+        Args:
+            avro_type: The Avro type, can be simple or complex
+
+        Returns:
+            The equivalent IcebergType
+
+        Raises:
+            ValueError: When there are unknown types
+        """
+        if isinstance(avro_type, str):
+            return PRIMITIVE_FIELD_TYPE_MAPPING[avro_type]
+        elif isinstance(avro_type, dict):
+            if "logicalType" in avro_type:
+                return self._convert_logical_type(avro_type)
+            else:
+                type_identifier = avro_type["type"]
+                if type_identifier == "record":
+                    return self._convert_record_type(avro_type)
+                elif type_identifier == "array":
+                    return self._convert_array_type(avro_type)
+                elif type_identifier == "map":
+                    return self._convert_map_type(avro_type)
+                elif type_identifier == "fixed":
+                    return self._convert_fixed_type(avro_type)
+                else:
+                    raise ValueError(f"Unknown type: {avro_type}")

Review Comment:
   It's uncommon, but it's valid for a primitive to be wrapped in a dict, like `{"type": "array", "items": {"type": "double", "additional": "property"}}`. That should be handled here. I think you can check whether `type_identifier` is in the primitive map.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r884326491


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,454 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Utility class for converting between Avro and Iceberg schemas
+
+"""
+from __future__ import annotations
+
+import logging
+from typing import Any, Dict, List, Tuple
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IcebergType,
+    IntegerType,
+    ListType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+    UUIDType,
+)
+
+logger = logging.getLogger(__name__)
+
+PRIMITIVE_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "boolean": BooleanType(),
+    "bytes": BinaryType(),
+    "double": DoubleType(),
+    "float": FloatType(),
+    "int": IntegerType(),
+    "long": LongType(),
+    "string": StringType(),
+    "enum": StringType(),
+}
+
+LOGICAL_FIELD_TYPE_MAPPING: Dict[Tuple[str, str], PrimitiveType] = {
+    ("date", "int"): DateType(),
+    ("time-millis", "int"): TimeType(),
+    ("timestamp-millis", "long"): TimestampType(),
+    ("time-micros", "int"): TimeType(),
+    ("timestamp-micros", "long"): TimestampType(),
+    ("uuid", "string"): UUIDType(),
+}
+
+
+class AvroSchemaConversion:
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expects to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into an Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+        """
+        return Schema(*[self._convert_field(field) for field in avro_schema["fields"]], schema_id=1)
+
+    def _resolve_union(self, type_union: Dict | List | str) -> Tuple[str | Dict[str, Any], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        Raises:
+            TypeError: In the case non-optional union types are encountered
+        """
+        avro_types: Dict | List
+        if isinstance(type_union, str):
+            # It is a primitive and required
+            return type_union, False
+        elif isinstance(type_union, dict):
+            # It is a context and required
+            return type_union, False
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        if len(avro_types) > 2:
+            raise TypeError("Non-optional types aren't part of the Iceberg specification")
+
+        # Filter the null value and return the type
+        return list(filter(lambda t: t != "null", avro_types))[0], is_optional
+
+    def _convert_schema(self, avro_type: str | Dict[str, Any]) -> IcebergType:
+        """
+        Resolves the Avro type
+
+        Args:
+            avro_type: The Avro type, can be simple or complex
+
+        Returns:
+            The equivalent IcebergType
+
+        Raises:
+            ValueError: When there are unknown types
+        """
+        if isinstance(avro_type, str):
+            return PRIMITIVE_FIELD_TYPE_MAPPING[avro_type]
+        elif isinstance(avro_type, dict):
+            if "logicalType" in avro_type:
+                return self._convert_logical_type(avro_type)
+            else:
+                # Resolve potential nested types
+                while "type" in avro_type and isinstance(avro_type["type"], dict):
+                    avro_type = avro_type["type"]
+                type_identifier = avro_type["type"]
+                if type_identifier == "record":
+                    return self._convert_record_type(avro_type)
+                elif type_identifier == "array":
+                    return self._convert_array_type(avro_type)
+                elif type_identifier == "map":
+                    return self._convert_map_type(avro_type)
+                elif type_identifier == "fixed":
+                    return self._convert_fixed_type(avro_type)
+                elif isinstance(type_identifier, str):
+                    return PRIMITIVE_FIELD_TYPE_MAPPING[type_identifier]
+                else:
+                    raise ValueError(f"Unknown type: {avro_type}")
+        else:
+            raise ValueError(f"Unknown type: {avro_type}")
+
+    def _convert_field(self, field: Dict[str, Any]) -> NestedField:
+        """
+        Converts an Avro field into an Iceberg equivalent field
+        Args:
+            field: The Avro field
+
+        Returns:
+            The Iceberg equivalent field
+        """
+        if "field-id" not in field:
+            raise ValueError(f"Cannot convert field, missing field-id: {field}")
+
+        plain_type, is_optional = self._resolve_union(field["type"])
+
+        return NestedField(
+            field_id=field["field-id"],
+            name=field["name"],
+            field_type=self._convert_schema(plain_type),
+            is_optional=is_optional,
+            doc=field.get("doc"),
+        )
+
+    def _convert_record_type(self, record_type: Dict[str, Any]) -> StructType:
+        """
+        Converts the fields from a record into an Iceberg struct
+
+        Examples:
+            >>> from iceberg.utils.schema_conversion import AvroSchemaConversion
+            >>> record_type = {
+            ...     "type": "record",
+            ...     "name": "r508",
+            ...     "fields": [{
+            ...         "name": "contains_null",
+            ...         "type": "boolean",
+            ...         "doc": "True if any file has a null partition value",
+            ...         "field-id": 509,
+            ...      }, {
+            ...          "name": "contains_nan",
+            ...          "type": ["null", "boolean"],
+            ...          "doc": "True if any file has a nan partition value",
+            ...          "default": None,
+            ...          "field-id": 518,
+            ...      }],
+            ... }
+            >>> actual = AvroSchemaConversion()._convert_record_type(record_type)
+            >>> expected = StructType(
+            ...     fields=(
+            ...         NestedField(
+            ...             field_id=509,
+            ...             name="contains_null",
+            ...             field_type=BooleanType(),
+            ...             is_optional=False,
+            ...             doc="True if any file has a null partition value",
+            ...         ),
+            ...         NestedField(
+            ...             field_id=518,
+            ...             name="contains_nan",
+            ...             field_type=BooleanType(),
+            ...             is_optional=True,
+            ...             doc="True if any file has a nan partition value",
+            ...         ),
+            ...     )
+            ... )
+            >>> expected == actual
+            True
+
+        Args:
+            record_type: The record type itself
+
+        Returns:
+        """
+        if record_type["type"] != "record":
+            raise ValueError(f"Expected type, got: {record_type}")
+
+        return StructType(*[self._convert_field(field) for field in record_type["fields"]])
+
+    def _convert_array_type(self, array_type: Dict[str, Any]) -> ListType:
+        if "element-id" not in array_type:
+            raise ValueError(f"Cannot convert array-type, missing element-id: {array_type}")
+
+        plain_type, element_is_optional = self._resolve_union(array_type["items"])
+
+        return ListType(
+            element_id=array_type["element-id"],
+            element_type=self._convert_schema(plain_type),
+            element_is_optional=element_is_optional,
+        )
+
+    def _convert_map_type(self, map_type: Dict[str, Any]) -> MapType:
+        """
+        Args:
+            map_type: The dict that describes the Avro map type
+
+        Examples:
+            >>> from iceberg.utils.schema_conversion import AvroSchemaConversion
+            >>> avro_field = {
+            ...     "type": "map",
+            ...     "values": ["long", "null"],
+            ...     "key-id": 101,
+            ...     "value-id": 102,
+            ... }
+            >>> actual = AvroSchemaConversion()._convert_map_type(avro_field)
+            >>> expected = MapType(
+            ...     key_id=101,
+            ...     key_type=StringType(),
+            ...     value_id=102,
+            ...     value_type=LongType(),
+            ...     value_is_optional=True
+            ... )
+            >>> actual == expected
+            True
+
+        Returns: A MapType
+        """
+        value_type, value_is_optional = self._resolve_union(map_type["values"])
+        return MapType(
+            key_id=map_type["key-id"],
+            # Avro only supports string keys
+            key_type=StringType(),
+            value_id=map_type["value-id"],
+            value_type=self._convert_schema(value_type),
+            value_is_optional=value_is_optional,
+        )
+
+    def _convert_logical_type(self, avro_logical_type: Dict[str, Any]) -> IcebergType:
+        """
+        Convert a schema with a logical type annotation. For the decimal and map
+        we need to fetch more keys from the dict, and for the simple ones we can just
+        look it up in the mapping.
+
+        Examples:
+            >>> from iceberg.utils.schema_conversion import AvroSchemaConversion
+            >>> avro_logical_type = {
+            ...     "type": "int",
+            ...     "logicalType": "date"
+            ... }
+            >>> actual = AvroSchemaConversion()._convert_logical_type(avro_logical_type)
+            >>> actual == DateType()
+            True
+
+        Args:
+            avro_logical_type: The logical type
+
+        Returns:
+            The converted logical type
+
+        Raises:
+            ValueError: When the logical type is unknown
+        """
+        logical_type = avro_logical_type["logicalType"]
+        physical_type = avro_logical_type["type"]
+        if logical_type == "decimal":
+            return self._convert_logical_decimal_type(avro_logical_type)
+        elif logical_type == "map":
+            return self._convert_logical_map_type(avro_logical_type)
+        elif (logical_type, physical_type) in LOGICAL_FIELD_TYPE_MAPPING:
+            return LOGICAL_FIELD_TYPE_MAPPING[(logical_type, physical_type)]
+        else:
+            raise ValueError(f"Unknown logical/physical type combination: {avro_logical_type}")
+
+    def _convert_logical_decimal_type(self, avro_type: Dict[str, Any]) -> DecimalType:
+        """
+        Args:
+            avro_type: The Avro type
+
+        Examples:
+            >>> from iceberg.utils.schema_conversion import AvroSchemaConversion
+            >>> avro_decimal_type = {
+            ...     "type": "bytes",
+            ...     "logicalType": "decimal",
+            ...     "precision": 19,
+            ...     "scale": 25
+            ... }
+            >>> actual = AvroSchemaConversion()._convert_logical_decimal_type(avro_decimal_type)
+            >>> expected = DecimalType(
+            ...     precision=19,
+            ...     scale=25
+            ... )
+            >>> actual == expected
+            True
+
+        Returns:
+            A Iceberg DecimalType
+        """
+        return DecimalType(precision=avro_type["precision"], scale=avro_type["scale"])
+
+    def _convert_logical_map_type(self, avro_type: Dict[str, Any]) -> MapType:
+        """
+        In the case where a map hasn't a key as a type you can use a logical map to
+        still encode this in Avro
+
+        Args:
+            avro_type: The Avro Type
+
+        Examples:
+            >>> from iceberg.utils.schema_conversion import AvroSchemaConversion
+            >>> avro_type = {
+            ...     "type": "array",
+            ...     "logicalType": "map",
+            ...     "items": {
+            ...         "type": "record",
+            ...         "name": "k101_v102",
+            ...         "fields": [
+            ...             {"name": "key", "type": "int", "field-id": 101},
+            ...             {"name": "value", "type": "string", "field-id": 102},
+            ...         ],
+            ...     },
+            ... }
+            >>> actual = AvroSchemaConversion()._convert_logical_map_type(avro_type)
+            >>> expected = MapType(
+            ...         key_id=101,
+            ...         key_type=IntegerType(),
+            ...         value_id=102,
+            ...         value_type=StringType(),
+            ...         value_is_optional=False
+            ... )
+            >>> actual == expected
+            True
+
+        .. _Apache Iceberg specification:
+            https://iceberg.apache.org/spec/#appendix-a-format-specific-requirements
+
+        Returns:
+            The logical map
+        """
+        fields = avro_type["items"]["fields"]
+        if len(fields) != 2:
+            raise ValueError(f'Invalid key-value pair schema: {avro_type["items"]}')
+        key = self._convert_field(list(filter(lambda f: f["name"] == "key", fields))[0])

Review Comment:
   Looks good. Too bad it is more complicated, but thanks for fixing 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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r884326452


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,454 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Utility class for converting between Avro and Iceberg schemas
+
+"""
+from __future__ import annotations
+
+import logging
+from typing import Any, Dict, List, Tuple
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IcebergType,
+    IntegerType,
+    ListType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+    UUIDType,
+)
+
+logger = logging.getLogger(__name__)
+
+PRIMITIVE_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "boolean": BooleanType(),
+    "bytes": BinaryType(),
+    "double": DoubleType(),
+    "float": FloatType(),
+    "int": IntegerType(),
+    "long": LongType(),
+    "string": StringType(),
+    "enum": StringType(),
+}
+
+LOGICAL_FIELD_TYPE_MAPPING: Dict[Tuple[str, str], PrimitiveType] = {
+    ("date", "int"): DateType(),
+    ("time-millis", "int"): TimeType(),
+    ("timestamp-millis", "long"): TimestampType(),
+    ("time-micros", "int"): TimeType(),
+    ("timestamp-micros", "long"): TimestampType(),
+    ("uuid", "string"): UUIDType(),
+}
+
+
+class AvroSchemaConversion:
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expects to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into an Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+        """
+        return Schema(*[self._convert_field(field) for field in avro_schema["fields"]], schema_id=1)
+
+    def _resolve_union(self, type_union: Dict | List | str) -> Tuple[str | Dict[str, Any], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        Raises:
+            TypeError: In the case non-optional union types are encountered
+        """
+        avro_types: Dict | List
+        if isinstance(type_union, str):
+            # It is a primitive and required
+            return type_union, False
+        elif isinstance(type_union, dict):
+            # It is a context and required
+            return type_union, False
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        if len(avro_types) > 2:
+            raise TypeError("Non-optional types aren't part of the Iceberg specification")
+
+        # Filter the null value and return the type
+        return list(filter(lambda t: t != "null", avro_types))[0], is_optional
+
+    def _convert_schema(self, avro_type: str | Dict[str, Any]) -> IcebergType:
+        """
+        Resolves the Avro type
+
+        Args:
+            avro_type: The Avro type, can be simple or complex
+
+        Returns:
+            The equivalent IcebergType
+
+        Raises:
+            ValueError: When there are unknown types
+        """
+        if isinstance(avro_type, str):
+            return PRIMITIVE_FIELD_TYPE_MAPPING[avro_type]
+        elif isinstance(avro_type, dict):
+            if "logicalType" in avro_type:
+                return self._convert_logical_type(avro_type)
+            else:
+                # Resolve potential nested types
+                while "type" in avro_type and isinstance(avro_type["type"], dict):
+                    avro_type = avro_type["type"]
+                type_identifier = avro_type["type"]
+                if type_identifier == "record":
+                    return self._convert_record_type(avro_type)
+                elif type_identifier == "array":
+                    return self._convert_array_type(avro_type)
+                elif type_identifier == "map":
+                    return self._convert_map_type(avro_type)
+                elif type_identifier == "fixed":
+                    return self._convert_fixed_type(avro_type)
+                elif isinstance(type_identifier, str):
+                    return PRIMITIVE_FIELD_TYPE_MAPPING[type_identifier]
+                else:
+                    raise ValueError(f"Unknown type: {avro_type}")
+        else:
+            raise ValueError(f"Unknown type: {avro_type}")
+
+    def _convert_field(self, field: Dict[str, Any]) -> NestedField:
+        """
+        Converts an Avro field into an Iceberg equivalent field
+        Args:
+            field: The Avro field
+
+        Returns:
+            The Iceberg equivalent field
+        """
+        if "field-id" not in field:
+            raise ValueError(f"Cannot convert field, missing field-id: {field}")
+
+        plain_type, is_optional = self._resolve_union(field["type"])
+
+        return NestedField(
+            field_id=field["field-id"],
+            name=field["name"],
+            field_type=self._convert_schema(plain_type),
+            is_optional=is_optional,
+            doc=field.get("doc"),
+        )
+
+    def _convert_record_type(self, record_type: Dict[str, Any]) -> StructType:
+        """
+        Converts the fields from a record into an Iceberg struct
+
+        Examples:
+            >>> from iceberg.utils.schema_conversion import AvroSchemaConversion
+            >>> record_type = {
+            ...     "type": "record",
+            ...     "name": "r508",
+            ...     "fields": [{
+            ...         "name": "contains_null",
+            ...         "type": "boolean",
+            ...         "doc": "True if any file has a null partition value",
+            ...         "field-id": 509,
+            ...      }, {
+            ...          "name": "contains_nan",
+            ...          "type": ["null", "boolean"],
+            ...          "doc": "True if any file has a nan partition value",
+            ...          "default": None,
+            ...          "field-id": 518,
+            ...      }],
+            ... }
+            >>> actual = AvroSchemaConversion()._convert_record_type(record_type)
+            >>> expected = StructType(
+            ...     fields=(
+            ...         NestedField(
+            ...             field_id=509,
+            ...             name="contains_null",
+            ...             field_type=BooleanType(),
+            ...             is_optional=False,
+            ...             doc="True if any file has a null partition value",
+            ...         ),
+            ...         NestedField(
+            ...             field_id=518,
+            ...             name="contains_nan",
+            ...             field_type=BooleanType(),
+            ...             is_optional=True,
+            ...             doc="True if any file has a nan partition value",
+            ...         ),
+            ...     )
+            ... )
+            >>> expected == actual
+            True
+
+        Args:
+            record_type: The record type itself
+
+        Returns:
+        """
+        if record_type["type"] != "record":
+            raise ValueError(f"Expected type, got: {record_type}")
+
+        return StructType(*[self._convert_field(field) for field in record_type["fields"]])
+
+    def _convert_array_type(self, array_type: Dict[str, Any]) -> ListType:
+        if "element-id" not in array_type:
+            raise ValueError(f"Cannot convert array-type, missing element-id: {array_type}")
+
+        plain_type, element_is_optional = self._resolve_union(array_type["items"])
+
+        return ListType(
+            element_id=array_type["element-id"],
+            element_type=self._convert_schema(plain_type),
+            element_is_optional=element_is_optional,
+        )
+
+    def _convert_map_type(self, map_type: Dict[str, Any]) -> MapType:
+        """
+        Args:
+            map_type: The dict that describes the Avro map type
+
+        Examples:
+            >>> from iceberg.utils.schema_conversion import AvroSchemaConversion
+            >>> avro_field = {
+            ...     "type": "map",
+            ...     "values": ["long", "null"],
+            ...     "key-id": 101,
+            ...     "value-id": 102,
+            ... }
+            >>> actual = AvroSchemaConversion()._convert_map_type(avro_field)
+            >>> expected = MapType(
+            ...     key_id=101,
+            ...     key_type=StringType(),
+            ...     value_id=102,
+            ...     value_type=LongType(),
+            ...     value_is_optional=True
+            ... )
+            >>> actual == expected
+            True
+
+        Returns: A MapType
+        """
+        value_type, value_is_optional = self._resolve_union(map_type["values"])
+        return MapType(
+            key_id=map_type["key-id"],
+            # Avro only supports string keys
+            key_type=StringType(),
+            value_id=map_type["value-id"],
+            value_type=self._convert_schema(value_type),
+            value_is_optional=value_is_optional,
+        )
+
+    def _convert_logical_type(self, avro_logical_type: Dict[str, Any]) -> IcebergType:
+        """
+        Convert a schema with a logical type annotation. For the decimal and map
+        we need to fetch more keys from the dict, and for the simple ones we can just
+        look it up in the mapping.
+
+        Examples:
+            >>> from iceberg.utils.schema_conversion import AvroSchemaConversion
+            >>> avro_logical_type = {
+            ...     "type": "int",
+            ...     "logicalType": "date"
+            ... }
+            >>> actual = AvroSchemaConversion()._convert_logical_type(avro_logical_type)
+            >>> actual == DateType()
+            True
+
+        Args:
+            avro_logical_type: The logical type
+
+        Returns:
+            The converted logical type
+
+        Raises:
+            ValueError: When the logical type is unknown
+        """
+        logical_type = avro_logical_type["logicalType"]
+        physical_type = avro_logical_type["type"]
+        if logical_type == "decimal":
+            return self._convert_logical_decimal_type(avro_logical_type)
+        elif logical_type == "map":
+            return self._convert_logical_map_type(avro_logical_type)
+        elif (logical_type, physical_type) in LOGICAL_FIELD_TYPE_MAPPING:
+            return LOGICAL_FIELD_TYPE_MAPPING[(logical_type, physical_type)]
+        else:
+            raise ValueError(f"Unknown logical/physical type combination: {avro_logical_type}")
+
+    def _convert_logical_decimal_type(self, avro_type: Dict[str, Any]) -> DecimalType:
+        """
+        Args:
+            avro_type: The Avro type
+
+        Examples:
+            >>> from iceberg.utils.schema_conversion import AvroSchemaConversion
+            >>> avro_decimal_type = {
+            ...     "type": "bytes",
+            ...     "logicalType": "decimal",
+            ...     "precision": 19,
+            ...     "scale": 25
+            ... }
+            >>> actual = AvroSchemaConversion()._convert_logical_decimal_type(avro_decimal_type)
+            >>> expected = DecimalType(
+            ...     precision=19,
+            ...     scale=25
+            ... )
+            >>> actual == expected
+            True
+
+        Returns:
+            A Iceberg DecimalType
+        """
+        return DecimalType(precision=avro_type["precision"], scale=avro_type["scale"])

Review Comment:
   Since you're validating physical type above, you may want to validate that the physical type here is fixed, long, or 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] rdblue merged pull request #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue merged PR #4742:
URL: https://github.com/apache/iceberg/pull/4742


-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r883694941


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,371 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Utility class for converting between Avro and Iceberg schemas
+
+"""
+import logging
+from typing import Any, Dict, List, Optional, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IcebergType,
+    IntegerType,
+    ListType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+    UUIDType,
+)
+
+logger = logging.getLogger(__name__)
+
+
+class AvroSchemaConversion:
+    PRIMITIVE_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "boolean": BooleanType(),
+        "bytes": BinaryType(),
+        "double": DoubleType(),
+        "float": FloatType(),
+        "int": IntegerType(),
+        "long": LongType(),
+        "string": StringType(),
+    }
+
+    LOGICAL_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "date": DateType(),
+        "time-millis": TimeType(),
+        "timestamp-millis": TimestampType(),
+        "uuid": UUIDType(),
+    }
+
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expects to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into a Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+        """
+        fields = self._parse_record(avro_schema)
+        return Schema(*fields.fields, schema_id=1)
+
+    def _parse_record(self, avro_field: Dict[str, Any]) -> StructType:
+        fields = [self._parse_field(field) for field in avro_field["fields"]]
+        return StructType(*fields)  # type: ignore
+
+    def _resolve_union(self, type_union: Union[Dict, List, str]) -> Tuple[Union[str, Dict[str, Any]], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        Raises:
+            TypeError: In the case non-optional union types are encountered
+        """
+        avro_types: Union[Dict, List]
+        if isinstance(type_union, str):
+            # It is a primitive and required
+            return type_union, False
+        elif isinstance(type_union, dict):
+            # It is a context and required
+            return type_union, False
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        # Filter the null value, so we know the actual type
+        avro_types = list(filter(lambda t: t != "null", avro_types))
+
+        if len(avro_types) != 1:
+            raise TypeError("Non-optional types aren't part of the Iceberg specification")
+
+        avro_type = avro_types[0]
+
+        return avro_type, is_optional
+
+    def _resolve_inner_type(
+        self, raw_avro_type: Dict[str, Any], inner_field_name: str, id_field: str
+    ) -> Tuple[IcebergType, bool]:
+        plain_type, element_is_optional = self._resolve_union(raw_avro_type[inner_field_name])
+        inner_field = self._parse_field(plain_type, field_id=raw_avro_type[id_field])
+        if isinstance(inner_field, NestedField):
+            return inner_field.field_type, element_is_optional
+        return inner_field, element_is_optional
+
+    def _parse_field(self, field: Union[str, Dict[str, Any]], field_id: Optional[int] = None) -> IcebergType:
+        """
+        Recursively walks through the Schema, constructing the Iceberg schema
+
+        Examples:
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {
+            ...             "name": "partitions",
+            ...                "type": [
+            ...                "null",
+            ...                {
+            ...                        "type": "array",
+            ...                        "items": {
+            ...                            "type": "record",
+            ...                            "name": "r508",
+            ...                            "fields": [
+            ...                                {
+            ...                                    "name": "contains_null",
+            ...                                    "type": "boolean",
+            ...                                    "doc": "True if any file has a null partition value",
+            ...                                    "field-id": 509,
+            ...                                },
+            ...                                {
+            ...                                    "name": "contains_nan",
+            ...                                    "type": ["null", "boolean"],
+            ...                                    "doc": "True if any file has a NaN partition value",
+            ...                                    "default": None,
+            ...                                    "field-id": 518,
+            ...                                },
+            ...                            ],
+            ...                        },
+            ...                        "element-id": 508,
+            ...                    },
+            ...                ],
+            ...                "doc": "Summary for each partition",
+            ...                "default": None,
+            ...                "field-id": 507,
+            ...            },
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=507,
+            ...         name="partitions",
+            ...         field_type=ListType(
+            ...             element_id=508,
+            ...             element_type=StructType(
+            ...                 NestedField(
+            ...                     field_id=509,
+            ...                     name="contains_null",
+            ...                     field_type=BooleanType(),
+            ...                     is_optional=False,
+            ...                     doc="True if any file has a null partition value",
+            ...                 ),
+            ...                 NestedField(
+            ...                     field_id=518,
+            ...                     name="contains_nan",
+            ...                     field_type=BooleanType(),
+            ...                     is_optional=True,
+            ...                     doc="True if any file has a NaN partition value",
+            ...                 )
+            ...             ),
+            ...             element_is_optional=False
+            ...         ),
+            ...         is_optional=True,
+            ...         doc="Summary for each partition",
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            field: The Avro field
+            field_id: Ability to override the field_id when it is provided from up in the tree (in the case of a list or map)
+
+        Returns:
+            The equivalent IcebergType
+        """
+        # In the case of a primitive field
+        if isinstance(field, str):
+            return AvroSchemaConversion.PRIMITIVE_FIELD_TYPE_MAP[field]

Review Comment:
   I find the logic in these methods hard to follow and I think it's because the cases are not cleanly separated by method. This is a good example. This method is handling an Avro field, but this is checking if the field is not a field and is instead a primitive type. That should never happen in a schema so it raises questions about when this method is called.
   
   This is one reason why we use the visitor pattern elsewhere. Keeping the logic to traverse a schema separate from the logic to actually do something with it is useful, but it also keeps you using a consistent and focused pattern to construct this logic: here's how to convert a record, here's how to convert a field, here's how to convert a map, etc.
   
   Since this isn't handling an Avro Schema class, I wasn't originally going to suggest it, but I think this would be cleaner and easier to review/maintain if it were structured around Avro's schema model:
   
   ```python
   Schema = 
   def _convert_schema(schema: str | Dict[str, Any]):
       if isinstance(schema, str):
           return AvroSchemaConversion.PRIMITIVE_FIELD_TYPE_MAP[schema]
       elif isinstance(schema, dict):
           type_name = schema["type"]
           if type_name == "record":
               return _convert_record(schema)
           elif type_name == "union":
               ...
           elif type_name == "map":
               ...
           elif type_name == "array":
               ...
           else:
               logical_type = schema.get("logicalType")
               if logicalType:
                   ...
               else:
                   return AvroSchemaConversion.PRIMITIVE_FIELD_TYPE_MAP[type_name]
       else:
           raise ValueError(f"Cannot convert invalid schema: {schema}")
   
   def _convert_record(schema: Dict[str, Any]):
       ... # calls _convert_field
   
   def _convert_field(field: Dict[str, Any]):
       ...
   ```
   
   



##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,371 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Utility class for converting between Avro and Iceberg schemas
+
+"""
+import logging
+from typing import Any, Dict, List, Optional, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IcebergType,
+    IntegerType,
+    ListType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+    UUIDType,
+)
+
+logger = logging.getLogger(__name__)
+
+
+class AvroSchemaConversion:
+    PRIMITIVE_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "boolean": BooleanType(),
+        "bytes": BinaryType(),
+        "double": DoubleType(),
+        "float": FloatType(),
+        "int": IntegerType(),
+        "long": LongType(),
+        "string": StringType(),
+    }
+
+    LOGICAL_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "date": DateType(),
+        "time-millis": TimeType(),
+        "timestamp-millis": TimestampType(),
+        "uuid": UUIDType(),
+    }
+
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expects to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into a Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+        """
+        fields = self._parse_record(avro_schema)
+        return Schema(*fields.fields, schema_id=1)
+
+    def _parse_record(self, avro_field: Dict[str, Any]) -> StructType:
+        fields = [self._parse_field(field) for field in avro_field["fields"]]
+        return StructType(*fields)  # type: ignore
+
+    def _resolve_union(self, type_union: Union[Dict, List, str]) -> Tuple[Union[str, Dict[str, Any]], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        Raises:
+            TypeError: In the case non-optional union types are encountered
+        """
+        avro_types: Union[Dict, List]
+        if isinstance(type_union, str):
+            # It is a primitive and required
+            return type_union, False
+        elif isinstance(type_union, dict):
+            # It is a context and required
+            return type_union, False
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        # Filter the null value, so we know the actual type
+        avro_types = list(filter(lambda t: t != "null", avro_types))
+
+        if len(avro_types) != 1:
+            raise TypeError("Non-optional types aren't part of the Iceberg specification")
+
+        avro_type = avro_types[0]
+
+        return avro_type, is_optional
+
+    def _resolve_inner_type(
+        self, raw_avro_type: Dict[str, Any], inner_field_name: str, id_field: str
+    ) -> Tuple[IcebergType, bool]:
+        plain_type, element_is_optional = self._resolve_union(raw_avro_type[inner_field_name])
+        inner_field = self._parse_field(plain_type, field_id=raw_avro_type[id_field])
+        if isinstance(inner_field, NestedField):
+            return inner_field.field_type, element_is_optional
+        return inner_field, element_is_optional
+
+    def _parse_field(self, field: Union[str, Dict[str, Any]], field_id: Optional[int] = None) -> IcebergType:
+        """
+        Recursively walks through the Schema, constructing the Iceberg schema
+
+        Examples:
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {
+            ...             "name": "partitions",
+            ...                "type": [
+            ...                "null",
+            ...                {
+            ...                        "type": "array",
+            ...                        "items": {
+            ...                            "type": "record",
+            ...                            "name": "r508",
+            ...                            "fields": [
+            ...                                {
+            ...                                    "name": "contains_null",
+            ...                                    "type": "boolean",
+            ...                                    "doc": "True if any file has a null partition value",
+            ...                                    "field-id": 509,
+            ...                                },
+            ...                                {
+            ...                                    "name": "contains_nan",
+            ...                                    "type": ["null", "boolean"],
+            ...                                    "doc": "True if any file has a NaN partition value",
+            ...                                    "default": None,
+            ...                                    "field-id": 518,
+            ...                                },
+            ...                            ],
+            ...                        },
+            ...                        "element-id": 508,
+            ...                    },
+            ...                ],
+            ...                "doc": "Summary for each partition",
+            ...                "default": None,
+            ...                "field-id": 507,
+            ...            },
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=507,
+            ...         name="partitions",
+            ...         field_type=ListType(
+            ...             element_id=508,
+            ...             element_type=StructType(
+            ...                 NestedField(
+            ...                     field_id=509,
+            ...                     name="contains_null",
+            ...                     field_type=BooleanType(),
+            ...                     is_optional=False,
+            ...                     doc="True if any file has a null partition value",
+            ...                 ),
+            ...                 NestedField(
+            ...                     field_id=518,
+            ...                     name="contains_nan",
+            ...                     field_type=BooleanType(),
+            ...                     is_optional=True,
+            ...                     doc="True if any file has a NaN partition value",
+            ...                 )
+            ...             ),
+            ...             element_is_optional=False
+            ...         ),
+            ...         is_optional=True,
+            ...         doc="Summary for each partition",
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            field: The Avro field
+            field_id: Ability to override the field_id when it is provided from up in the tree (in the case of a list or map)
+
+        Returns:
+            The equivalent IcebergType
+        """
+        # In the case of a primitive field
+        if isinstance(field, str):
+            return AvroSchemaConversion.PRIMITIVE_FIELD_TYPE_MAP[field]

Review Comment:
   I find the logic in these methods hard to follow and I think it's because the cases are not cleanly separated by method. This is a good example. This method is handling an Avro field, but this is checking if the field is not a field and is instead a primitive type. That should never happen in a schema so it raises questions about when this method is called.
   
   This is one reason why we use the visitor pattern elsewhere. Keeping the logic to traverse a schema separate from the logic to actually do something with it is useful, but it also keeps you using a consistent and focused pattern to construct this logic: here's how to convert a record, here's how to convert a field, here's how to convert a map, etc.
   
   Since this isn't handling an Avro Schema class, I wasn't originally going to suggest it, but I think this would be cleaner and easier to review/maintain if it were structured around Avro's schema model:
   
   ```python
   def _convert_schema(schema: str | Dict[str, Any]):
       if isinstance(schema, str):
           return AvroSchemaConversion.PRIMITIVE_FIELD_TYPE_MAP[schema]
       elif isinstance(schema, dict):
           type_name = schema["type"]
           if type_name == "record":
               return _convert_record(schema)
           elif type_name == "union":
               ...
           elif type_name == "map":
               ...
           elif type_name == "array":
               ...
           else:
               logical_type = schema.get("logicalType")
               if logicalType:
                   ...
               else:
                   return AvroSchemaConversion.PRIMITIVE_FIELD_TYPE_MAP[type_name]
       else:
           raise ValueError(f"Cannot convert invalid schema: {schema}")
   
   def _convert_record(schema: Dict[str, Any]):
       ... # calls _convert_field
   
   def _convert_field(field: Dict[str, Any]):
       ...
   ```
   
   



-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
Fokko commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r883919490


##########
python/src/iceberg/schema.py:
##########
@@ -67,6 +67,17 @@ def __repr__(self):
             f"Schema(fields={repr(self.columns)}, schema_id={self.schema_id}, identifier_field_ids={self.identifier_field_ids})"
         )
 
+    def __eq__(self, other) -> bool:

Review Comment:
   I was a bit too quick on this one. Schema doesn't have an autogenerated one, but the types do have one now. So we still need the eq on the schema



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r883234778


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,371 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Utility class for converting between Avro and Iceberg schemas
+
+"""
+import logging
+from typing import Any, Dict, List, Optional, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IcebergType,
+    IntegerType,
+    ListType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+    UUIDType,
+)
+
+logger = logging.getLogger(__name__)
+
+
+class AvroSchemaConversion:
+    PRIMITIVE_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "boolean": BooleanType(),
+        "bytes": BinaryType(),
+        "double": DoubleType(),
+        "float": FloatType(),
+        "int": IntegerType(),
+        "long": LongType(),
+        "string": StringType(),
+    }
+
+    LOGICAL_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "date": DateType(),
+        "time-millis": TimeType(),
+        "timestamp-millis": TimestampType(),
+        "uuid": UUIDType(),
+    }
+
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expects to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into a Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+        """
+        fields = self._parse_record(avro_schema)
+        return Schema(*fields.fields, schema_id=1)
+
+    def _parse_record(self, avro_field: Dict[str, Any]) -> StructType:
+        fields = [self._parse_field(field) for field in avro_field["fields"]]

Review Comment:
   Should this check that the Avro schema's `type` is `record`? That seems safer than assuming that `fields` is a key.



-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
Fokko commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r876276330


##########
python/src/iceberg/schema.py:
##########
@@ -67,6 +67,17 @@ def __repr__(self):
             f"Schema(fields={repr(self.columns)}, schema_id={self.schema_id}, identifier_field_ids={self.identifier_field_ids})"
         )
 
+    def __eq__(self, other) -> bool:

Review Comment:
   This is now also implemented using the autogenerated `__eq__` of the dataclass 👍🏻 



-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
Fokko commented on PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#issuecomment-1132269896

   @rdblue @kbendick I've updated the PR with the comments addressed 👍🏻 


-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
Fokko commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r884075163


##########
python/src/iceberg/schema.py:
##########
@@ -67,6 +67,17 @@ def __repr__(self):
             f"Schema(fields={repr(self.columns)}, schema_id={self.schema_id}, identifier_field_ids={self.identifier_field_ids})"
         )
 
+    def __eq__(self, other) -> bool:

Review Comment:
   Ah, right! Sorry about that



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r884296965


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,451 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Utility class for converting between Avro and Iceberg schemas
+
+"""
+from __future__ import annotations
+
+import logging
+from typing import Any, Dict, List, Tuple
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IcebergType,
+    IntegerType,
+    ListType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+    UUIDType,
+)
+
+logger = logging.getLogger(__name__)
+
+PRIMITIVE_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "boolean": BooleanType(),
+    "bytes": BinaryType(),
+    "double": DoubleType(),
+    "float": FloatType(),
+    "int": IntegerType(),
+    "long": LongType(),
+    "string": StringType(),
+}
+
+LOGICAL_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "date": DateType(),
+    "time-millis": TimeType(),
+    "timestamp-millis": TimestampType(),
+    "uuid": UUIDType(),
+}
+
+
+class AvroSchemaConversion:
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expects to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into an Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+        """
+        return Schema(*[self._convert_field(field) for field in avro_schema["fields"]], schema_id=1)
+
+    def _resolve_union(self, type_union: Dict | List | str) -> Tuple[str | Dict[str, Any], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        Raises:
+            TypeError: In the case non-optional union types are encountered
+        """
+        avro_types: Dict | List
+        if isinstance(type_union, str):
+            # It is a primitive and required
+            return type_union, False
+        elif isinstance(type_union, dict):
+            # It is a context and required
+            return type_union, False
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        if len(avro_types) > 2:
+            raise TypeError("Non-optional types aren't part of the Iceberg specification")
+
+        # Filter the null value and return the type
+        return list(filter(lambda t: t != "null", avro_types))[0], is_optional
+
+    def _resolve_inner_type(self, avro_type: Dict[str, Any], inner_field_name: str, id_field: str) -> Tuple[IcebergType, bool]:
+        """
+        Removes any unions inside an array or map, and appends the field-id downstream (if required)
+
+        Args:
+            avro_type: The parent Avro type
+            inner_field_name: For example 'items' in case of an array
+            id_field: Contains the element or key/value id, for example element-id, to be copied to the field-id
+
+        Returns:
+            The inner type and the nullability
+        """
+        plain_type, element_is_optional = self._resolve_union(avro_type[inner_field_name])
+        if isinstance(plain_type, dict):
+            # We need the element-id downstream
+            plain_type["field-id"] = avro_type[id_field]

Review Comment:
   I strongly prefer not modifying the incoming data like this is doing. I think `avro_type` and parts of that type should be handled as though they are immutable.
   
   Also, I don't think that this works. `_convert_schema` is going to return a type. For primitive types, there is no way to pass in the `field-id` property, and in all cases the `field-id` is going to be ignored because Iceberg types don't carry field IDs.
   
   I think the simple solution to both problems is to handle inner types in the map and array methods directly, rather than using a common method.



-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
Fokko commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r884315249


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,451 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Utility class for converting between Avro and Iceberg schemas
+
+"""
+from __future__ import annotations
+
+import logging
+from typing import Any, Dict, List, Tuple
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IcebergType,
+    IntegerType,
+    ListType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+    UUIDType,
+)
+
+logger = logging.getLogger(__name__)
+
+PRIMITIVE_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "boolean": BooleanType(),
+    "bytes": BinaryType(),
+    "double": DoubleType(),
+    "float": FloatType(),
+    "int": IntegerType(),
+    "long": LongType(),
+    "string": StringType(),
+}
+
+LOGICAL_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "date": DateType(),
+    "time-millis": TimeType(),
+    "timestamp-millis": TimestampType(),
+    "uuid": UUIDType(),
+}
+
+
+class AvroSchemaConversion:
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expects to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into an Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+        """
+        return Schema(*[self._convert_field(field) for field in avro_schema["fields"]], schema_id=1)
+
+    def _resolve_union(self, type_union: Dict | List | str) -> Tuple[str | Dict[str, Any], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        Raises:
+            TypeError: In the case non-optional union types are encountered
+        """
+        avro_types: Dict | List
+        if isinstance(type_union, str):
+            # It is a primitive and required
+            return type_union, False
+        elif isinstance(type_union, dict):
+            # It is a context and required
+            return type_union, False
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        if len(avro_types) > 2:
+            raise TypeError("Non-optional types aren't part of the Iceberg specification")
+
+        # Filter the null value and return the type
+        return list(filter(lambda t: t != "null", avro_types))[0], is_optional
+
+    def _resolve_inner_type(self, avro_type: Dict[str, Any], inner_field_name: str, id_field: str) -> Tuple[IcebergType, bool]:
+        """
+        Removes any unions inside an array or map, and appends the field-id downstream (if required)
+
+        Args:
+            avro_type: The parent Avro type
+            inner_field_name: For example 'items' in case of an array
+            id_field: Contains the element or key/value id, for example element-id, to be copied to the field-id
+
+        Returns:
+            The inner type and the nullability
+        """
+        plain_type, element_is_optional = self._resolve_union(avro_type[inner_field_name])
+        if isinstance(plain_type, dict):
+            # We need the element-id downstream
+            plain_type["field-id"] = avro_type[id_field]
+        inner_field = self._convert_schema(plain_type)
+        return inner_field, element_is_optional
+
+    def _convert_schema(self, avro_type: str | Dict[str, Any]) -> IcebergType:
+        """
+        Resolves the Avro type
+
+        Args:
+            avro_type: The Avro type, can be simple or complex
+
+        Returns:
+            The equivalent IcebergType
+
+        Raises:
+            ValueError: When there are unknown types
+        """
+        if isinstance(avro_type, str):
+            return PRIMITIVE_FIELD_TYPE_MAPPING[avro_type]
+        elif isinstance(avro_type, dict):
+            if "logicalType" in avro_type:
+                return self._convert_logical_type(avro_type)
+            else:
+                type_identifier = avro_type["type"]
+                if type_identifier == "record":
+                    return self._convert_record_type(avro_type)
+                elif type_identifier == "array":
+                    return self._convert_array_type(avro_type)
+                elif type_identifier == "map":
+                    return self._convert_map_type(avro_type)
+                elif type_identifier == "fixed":
+                    return self._convert_fixed_type(avro_type)
+                else:
+                    raise ValueError(f"Unknown type: {avro_type}")
+        else:
+            raise ValueError(f"Unknown type: {avro_type}")
+
+    def _convert_field(self, field: Dict[str, Any]) -> NestedField:
+        """
+        Converts an Avro field into an Iceberg equivalent field
+        Args:
+            field: The Avro field
+
+        Returns:
+            The Iceberg equivalent field
+        """
+        assert "field-id" in field, "Missing field-id in the Avro field, this is required for converting it to an Iceberg schema"

Review Comment:
   Great point. I'm not against assert's personally, but there is a battle in the Python community if you should use them for checks like these. Technically you can ignore them using a flag `python -O`, and they are meant for developers. I've changed them to a ValueError to be in line with the rest of the exceptions (which I like).
   
   I've updated the message to your suggestion. I think it is good to have the field in there 👍🏻 



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r884298128


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,451 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Utility class for converting between Avro and Iceberg schemas
+
+"""
+from __future__ import annotations
+
+import logging
+from typing import Any, Dict, List, Tuple
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IcebergType,
+    IntegerType,
+    ListType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+    UUIDType,
+)
+
+logger = logging.getLogger(__name__)
+
+PRIMITIVE_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "boolean": BooleanType(),
+    "bytes": BinaryType(),
+    "double": DoubleType(),
+    "float": FloatType(),
+    "int": IntegerType(),
+    "long": LongType(),
+    "string": StringType(),
+}
+
+LOGICAL_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "date": DateType(),
+    "time-millis": TimeType(),
+    "timestamp-millis": TimestampType(),

Review Comment:
   Iceberg will only write `time-micros` and `timestamp-micros` because at least microsecond precision is required by the SQL spec. The mappings for millis are correct, but we should also be able to handle micros.



-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r884326146


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,454 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Utility class for converting between Avro and Iceberg schemas
+
+"""
+from __future__ import annotations
+
+import logging
+from typing import Any, Dict, List, Tuple
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IcebergType,
+    IntegerType,
+    ListType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+    UUIDType,
+)
+
+logger = logging.getLogger(__name__)
+
+PRIMITIVE_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "boolean": BooleanType(),
+    "bytes": BinaryType(),
+    "double": DoubleType(),
+    "float": FloatType(),
+    "int": IntegerType(),
+    "long": LongType(),
+    "string": StringType(),
+    "enum": StringType(),
+}
+
+LOGICAL_FIELD_TYPE_MAPPING: Dict[Tuple[str, str], PrimitiveType] = {
+    ("date", "int"): DateType(),
+    ("time-millis", "int"): TimeType(),
+    ("timestamp-millis", "long"): TimestampType(),
+    ("time-micros", "int"): TimeType(),
+    ("timestamp-micros", "long"): TimestampType(),
+    ("uuid", "string"): UUIDType(),
+}
+
+
+class AvroSchemaConversion:
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expects to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into an Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+        """
+        return Schema(*[self._convert_field(field) for field in avro_schema["fields"]], schema_id=1)
+
+    def _resolve_union(self, type_union: Dict | List | str) -> Tuple[str | Dict[str, Any], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        Raises:
+            TypeError: In the case non-optional union types are encountered
+        """
+        avro_types: Dict | List
+        if isinstance(type_union, str):
+            # It is a primitive and required
+            return type_union, False
+        elif isinstance(type_union, dict):
+            # It is a context and required
+            return type_union, False
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        if len(avro_types) > 2:
+            raise TypeError("Non-optional types aren't part of the Iceberg specification")
+
+        # Filter the null value and return the type
+        return list(filter(lambda t: t != "null", avro_types))[0], is_optional
+
+    def _convert_schema(self, avro_type: str | Dict[str, Any]) -> IcebergType:
+        """
+        Resolves the Avro type
+
+        Args:
+            avro_type: The Avro type, can be simple or complex
+
+        Returns:
+            The equivalent IcebergType
+
+        Raises:
+            ValueError: When there are unknown types
+        """
+        if isinstance(avro_type, str):
+            return PRIMITIVE_FIELD_TYPE_MAPPING[avro_type]
+        elif isinstance(avro_type, dict):
+            if "logicalType" in avro_type:
+                return self._convert_logical_type(avro_type)
+            else:
+                # Resolve potential nested types
+                while "type" in avro_type and isinstance(avro_type["type"], dict):
+                    avro_type = avro_type["type"]
+                type_identifier = avro_type["type"]
+                if type_identifier == "record":
+                    return self._convert_record_type(avro_type)
+                elif type_identifier == "array":
+                    return self._convert_array_type(avro_type)
+                elif type_identifier == "map":
+                    return self._convert_map_type(avro_type)
+                elif type_identifier == "fixed":
+                    return self._convert_fixed_type(avro_type)
+                elif isinstance(type_identifier, str):
+                    return PRIMITIVE_FIELD_TYPE_MAPPING[type_identifier]
+                else:
+                    raise ValueError(f"Unknown type: {avro_type}")
+        else:
+            raise ValueError(f"Unknown type: {avro_type}")
+
+    def _convert_field(self, field: Dict[str, Any]) -> NestedField:
+        """
+        Converts an Avro field into an Iceberg equivalent field
+        Args:
+            field: The Avro field
+
+        Returns:
+            The Iceberg equivalent field
+        """
+        if "field-id" not in field:
+            raise ValueError(f"Cannot convert field, missing field-id: {field}")
+
+        plain_type, is_optional = self._resolve_union(field["type"])
+
+        return NestedField(
+            field_id=field["field-id"],
+            name=field["name"],
+            field_type=self._convert_schema(plain_type),
+            is_optional=is_optional,
+            doc=field.get("doc"),
+        )
+
+    def _convert_record_type(self, record_type: Dict[str, Any]) -> StructType:
+        """
+        Converts the fields from a record into an Iceberg struct
+
+        Examples:
+            >>> from iceberg.utils.schema_conversion import AvroSchemaConversion
+            >>> record_type = {
+            ...     "type": "record",
+            ...     "name": "r508",
+            ...     "fields": [{
+            ...         "name": "contains_null",
+            ...         "type": "boolean",
+            ...         "doc": "True if any file has a null partition value",
+            ...         "field-id": 509,
+            ...      }, {
+            ...          "name": "contains_nan",
+            ...          "type": ["null", "boolean"],
+            ...          "doc": "True if any file has a nan partition value",
+            ...          "default": None,
+            ...          "field-id": 518,
+            ...      }],
+            ... }
+            >>> actual = AvroSchemaConversion()._convert_record_type(record_type)
+            >>> expected = StructType(
+            ...     fields=(
+            ...         NestedField(
+            ...             field_id=509,
+            ...             name="contains_null",
+            ...             field_type=BooleanType(),
+            ...             is_optional=False,
+            ...             doc="True if any file has a null partition value",
+            ...         ),
+            ...         NestedField(
+            ...             field_id=518,
+            ...             name="contains_nan",
+            ...             field_type=BooleanType(),
+            ...             is_optional=True,
+            ...             doc="True if any file has a nan partition value",
+            ...         ),
+            ...     )
+            ... )
+            >>> expected == actual
+            True
+
+        Args:
+            record_type: The record type itself
+
+        Returns:
+        """
+        if record_type["type"] != "record":
+            raise ValueError(f"Expected type, got: {record_type}")

Review Comment:
   Expected record?



-- 
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] kbendick commented on a diff in pull request #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r869494393


##########
python/src/iceberg/schema.py:
##########
@@ -67,6 +67,17 @@ def __repr__(self):
             f"Schema(fields={repr(self.columns)}, schema_id={self.schema_id}, identifier_field_ids={self.identifier_field_ids})"
         )
 
+    def __eq__(self, other) -> bool:
+        if not other:
+            return False
+
+        columns = list(self.columns)
+
+        if len(columns) != len(other.columns):
+            return False
+
+        return all([lhs == rhs for lhs, rhs in zip(columns, other.columns)])

Review Comment:
   Question: Should we be comparing the `NestedField` based on id as well? I _believe_ we're ok to just assume that they're in order, but good to check. Here's the Java equals for `NestedField`:
   
   https://github.com/apache/iceberg/blob/06cdfa3f7c4ebd26766bc67100ba5d30b6bf3940/api/src/main/java/org/apache/iceberg/types/Types.java#L502-L521



-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r872839779


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,274 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Util class for converting between Avro and Iceberg schemas
+
+"""
+import logging
+from typing import Any, Dict, List, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DoubleType,
+    FloatType,
+    IntegerType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+)
+
+logger = logging.getLogger(__name__)
+
+
+class AvroSchemaConversion:
+    PRIMITIVE_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "boolean": BooleanType(),
+        "bytes": BinaryType(),
+        "date": DateType(),
+        "double": DoubleType(),
+        "float": FloatType(),
+        "int": IntegerType(),
+        "long": LongType(),
+        "string": StringType(),
+        "time-millis": TimeType(),
+        "timestamp-millis": TimestampType(),
+    }
+
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expect to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into a Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+
+        Todo:
+            * Implement full support for unions
+            * Implement logical types
+        """
+        fields = self._parse_record(avro_schema)
+        return Schema(*fields.fields, schema_id=1)  # type: ignore
+
+    def _parse_record(self, avro_field: Dict[str, Any]) -> StructType:
+        return StructType(*[self._parse_field(field) for field in avro_field["fields"]])
+
+    def _resolve_union(self, type_union: Union[Dict, List, str]) -> Tuple[Union[str, dict], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        """
+        avro_types: Union[Dict, List]
+        if isinstance(type_union, str):
+            avro_types = [type_union]
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        # Filter the null value, so we know the actual type
+        avro_types = list(filter(lambda t: t != "null", avro_types))
+
+        if len(avro_types) != 1:
+            raise ValueError("Support for unions is yet to be implemented")
+
+        avro_type = avro_types[0]
+
+        return avro_type, is_optional
+
+    def _parse_field(self, field: Dict[str, Any]) -> NestedField:
+        """
+        Recursively walks through the Schema, constructing the Iceberg schema
+
+        Examples:
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {
+            ...             "name": "partitions",
+            ...                "type": [
+            ...                "null",
+            ...                {
+            ...                        "type": "array",
+            ...                        "items": {
+            ...                            "type": "record",
+            ...                            "name": "r508",
+            ...                            "fields": [
+            ...                                {
+            ...                                    "name": "contains_null",
+            ...                                    "type": "boolean",
+            ...                                    "doc": "True if any file has a null partition value",
+            ...                                    "field-id": 509,
+            ...                                },
+            ...                                {
+            ...                                    "name": "contains_nan",
+            ...                                    "type": ["null", "boolean"],
+            ...                                    "doc": "True if any file has a nan partition value",
+            ...                                    "default": None,
+            ...                                    "field-id": 518,
+            ...                                },
+            ...                            ],
+            ...                        },
+            ...                        "element-id": 508,
+            ...                    },
+            ...                ],
+            ...                "doc": "Summary for each partition",
+            ...                "default": None,
+            ...                "field-id": 507,
+            ...            },
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=507,
+            ...         name="partitions",
+            ...         field_type=StructType(
+            ...             NestedField(
+            ...                 field_id=509,
+            ...                 name="contains_null",
+            ...                 field_type=BooleanType(),
+            ...                 is_optional=False,
+            ...                 doc="True if any file has a null partition value",
+            ...             ),
+            ...             NestedField(
+            ...                 field_id=518,
+            ...                 name="contains_nan",
+            ...                 field_type=BooleanType(),
+            ...                 is_optional=True,
+            ...                 doc="True if any file has a nan partition value",
+            ...             ),
+            ...         ),
+            ...         is_optional=True,
+            ...         doc="Summary for each partition",
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            field:
+
+        Returns:
+
+        """
+        field_id = field["field-id"]
+        field_name = field["name"]
+        field_doc = field.get("doc")
+
+        avro_type, is_optional = self._resolve_union(field["type"])
+        if isinstance(avro_type, dict):
+            avro_type = avro_type["type"]
+
+        if isinstance(avro_type, str) and avro_type in AvroSchemaConversion.PRIMITIVE_FIELD_TYPE_MAP:
+            return NestedField(
+                field_id=field_id,
+                name=field_name,
+                field_type=AvroSchemaConversion.PRIMITIVE_FIELD_TYPE_MAP[avro_type],
+                is_optional=is_optional,
+                doc=field_doc,
+            )
+        elif avro_type == "record":
+            return NestedField(
+                field_id=field_id, name=field_name, field_type=self._parse_record(field), is_optional=is_optional, doc=field_doc
+            )
+        elif avro_type == "array":
+            inner_type, _ = self._resolve_union(field["type"])
+            assert isinstance(inner_type, dict)
+            inner_type["items"]["field-id"] = inner_type["element-id"]

Review Comment:
   I don't think there is a guarantee that `"items"` returns a `dict`. The example from the Avro spec is `{ "type": "array", "items": "string", "default": [] }` so we know `"items"` can be a string.
   
   I think the motivation is to be able to call `_parse_field` on the result, but you could also just create the `NestedField` here after converting the type.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r872839779


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,274 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Util class for converting between Avro and Iceberg schemas
+
+"""
+import logging
+from typing import Any, Dict, List, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DoubleType,
+    FloatType,
+    IntegerType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+)
+
+logger = logging.getLogger(__name__)
+
+
+class AvroSchemaConversion:
+    PRIMITIVE_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "boolean": BooleanType(),
+        "bytes": BinaryType(),
+        "date": DateType(),
+        "double": DoubleType(),
+        "float": FloatType(),
+        "int": IntegerType(),
+        "long": LongType(),
+        "string": StringType(),
+        "time-millis": TimeType(),
+        "timestamp-millis": TimestampType(),
+    }
+
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expect to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into a Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+
+        Todo:
+            * Implement full support for unions
+            * Implement logical types
+        """
+        fields = self._parse_record(avro_schema)
+        return Schema(*fields.fields, schema_id=1)  # type: ignore
+
+    def _parse_record(self, avro_field: Dict[str, Any]) -> StructType:
+        return StructType(*[self._parse_field(field) for field in avro_field["fields"]])
+
+    def _resolve_union(self, type_union: Union[Dict, List, str]) -> Tuple[Union[str, dict], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        """
+        avro_types: Union[Dict, List]
+        if isinstance(type_union, str):
+            avro_types = [type_union]
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        # Filter the null value, so we know the actual type
+        avro_types = list(filter(lambda t: t != "null", avro_types))
+
+        if len(avro_types) != 1:
+            raise ValueError("Support for unions is yet to be implemented")
+
+        avro_type = avro_types[0]
+
+        return avro_type, is_optional
+
+    def _parse_field(self, field: Dict[str, Any]) -> NestedField:
+        """
+        Recursively walks through the Schema, constructing the Iceberg schema
+
+        Examples:
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {
+            ...             "name": "partitions",
+            ...                "type": [
+            ...                "null",
+            ...                {
+            ...                        "type": "array",
+            ...                        "items": {
+            ...                            "type": "record",
+            ...                            "name": "r508",
+            ...                            "fields": [
+            ...                                {
+            ...                                    "name": "contains_null",
+            ...                                    "type": "boolean",
+            ...                                    "doc": "True if any file has a null partition value",
+            ...                                    "field-id": 509,
+            ...                                },
+            ...                                {
+            ...                                    "name": "contains_nan",
+            ...                                    "type": ["null", "boolean"],
+            ...                                    "doc": "True if any file has a nan partition value",
+            ...                                    "default": None,
+            ...                                    "field-id": 518,
+            ...                                },
+            ...                            ],
+            ...                        },
+            ...                        "element-id": 508,
+            ...                    },
+            ...                ],
+            ...                "doc": "Summary for each partition",
+            ...                "default": None,
+            ...                "field-id": 507,
+            ...            },
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=507,
+            ...         name="partitions",
+            ...         field_type=StructType(
+            ...             NestedField(
+            ...                 field_id=509,
+            ...                 name="contains_null",
+            ...                 field_type=BooleanType(),
+            ...                 is_optional=False,
+            ...                 doc="True if any file has a null partition value",
+            ...             ),
+            ...             NestedField(
+            ...                 field_id=518,
+            ...                 name="contains_nan",
+            ...                 field_type=BooleanType(),
+            ...                 is_optional=True,
+            ...                 doc="True if any file has a nan partition value",
+            ...             ),
+            ...         ),
+            ...         is_optional=True,
+            ...         doc="Summary for each partition",
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            field:
+
+        Returns:
+
+        """
+        field_id = field["field-id"]
+        field_name = field["name"]
+        field_doc = field.get("doc")
+
+        avro_type, is_optional = self._resolve_union(field["type"])
+        if isinstance(avro_type, dict):
+            avro_type = avro_type["type"]
+
+        if isinstance(avro_type, str) and avro_type in AvroSchemaConversion.PRIMITIVE_FIELD_TYPE_MAP:
+            return NestedField(
+                field_id=field_id,
+                name=field_name,
+                field_type=AvroSchemaConversion.PRIMITIVE_FIELD_TYPE_MAP[avro_type],
+                is_optional=is_optional,
+                doc=field_doc,
+            )
+        elif avro_type == "record":
+            return NestedField(
+                field_id=field_id, name=field_name, field_type=self._parse_record(field), is_optional=is_optional, doc=field_doc
+            )
+        elif avro_type == "array":
+            inner_type, _ = self._resolve_union(field["type"])
+            assert isinstance(inner_type, dict)
+            inner_type["items"]["field-id"] = inner_type["element-id"]

Review Comment:
   I don't think there is a guarantee that `"items"` returns a `dict`. The example from the Avro spec is `{ "type": "array", "items": "string", "default": [] }` so we know `"items"` can be a string.
   
   I think you want to set `field_id` instead since that's what you use when constructing the `NestedField`.



-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
Fokko commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r876239306


##########
python/src/iceberg/schema.py:
##########
@@ -67,6 +67,17 @@ def __repr__(self):
             f"Schema(fields={repr(self.columns)}, schema_id={self.schema_id}, identifier_field_ids={self.identifier_field_ids})"
         )
 
+    def __eq__(self, other) -> bool:
+        if not other:
+            return False
+
+        columns = list(self.columns)
+
+        if len(columns) != len(other.columns):
+            return False
+
+        return all([lhs == rhs for lhs, rhs in zip(columns, other.columns)])

Review Comment:
   Great point @kbendick. Now https://github.com/apache/iceberg/pull/4767 has been merged, the eq methods are generated automatically :)



-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
Fokko commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r876281571


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,274 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Util class for converting between Avro and Iceberg schemas
+
+"""
+import logging
+from typing import Any, Dict, List, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DoubleType,
+    FloatType,
+    IntegerType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+)
+
+logger = logging.getLogger(__name__)
+
+
+class AvroSchemaConversion:
+    PRIMITIVE_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "boolean": BooleanType(),
+        "bytes": BinaryType(),
+        "date": DateType(),
+        "double": DoubleType(),
+        "float": FloatType(),
+        "int": IntegerType(),
+        "long": LongType(),
+        "string": StringType(),
+        "time-millis": TimeType(),
+        "timestamp-millis": TimestampType(),
+    }
+
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expect to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into a Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+
+        Todo:
+            * Implement full support for unions
+            * Implement logical types
+        """
+        fields = self._parse_record(avro_schema)
+        return Schema(*fields.fields, schema_id=1)  # type: ignore
+
+    def _parse_record(self, avro_field: Dict[str, Any]) -> StructType:
+        return StructType(*[self._parse_field(field) for field in avro_field["fields"]])
+
+    def _resolve_union(self, type_union: Union[Dict, List, str]) -> Tuple[Union[str, dict], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        """
+        avro_types: Union[Dict, List]
+        if isinstance(type_union, str):
+            avro_types = [type_union]
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        # Filter the null value, so we know the actual type
+        avro_types = list(filter(lambda t: t != "null", avro_types))
+
+        if len(avro_types) != 1:
+            raise ValueError("Support for unions is yet to be implemented")
+
+        avro_type = avro_types[0]
+
+        return avro_type, is_optional
+
+    def _parse_field(self, field: Dict[str, Any]) -> NestedField:
+        """
+        Recursively walks through the Schema, constructing the Iceberg schema
+
+        Examples:
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {
+            ...             "name": "partitions",
+            ...                "type": [
+            ...                "null",
+            ...                {
+            ...                        "type": "array",
+            ...                        "items": {
+            ...                            "type": "record",
+            ...                            "name": "r508",
+            ...                            "fields": [
+            ...                                {
+            ...                                    "name": "contains_null",
+            ...                                    "type": "boolean",
+            ...                                    "doc": "True if any file has a null partition value",
+            ...                                    "field-id": 509,
+            ...                                },
+            ...                                {
+            ...                                    "name": "contains_nan",
+            ...                                    "type": ["null", "boolean"],
+            ...                                    "doc": "True if any file has a nan partition value",
+            ...                                    "default": None,
+            ...                                    "field-id": 518,
+            ...                                },
+            ...                            ],
+            ...                        },
+            ...                        "element-id": 508,
+            ...                    },
+            ...                ],
+            ...                "doc": "Summary for each partition",
+            ...                "default": None,
+            ...                "field-id": 507,
+            ...            },
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=507,
+            ...         name="partitions",
+            ...         field_type=StructType(
+            ...             NestedField(
+            ...                 field_id=509,
+            ...                 name="contains_null",
+            ...                 field_type=BooleanType(),
+            ...                 is_optional=False,
+            ...                 doc="True if any file has a null partition value",
+            ...             ),
+            ...             NestedField(
+            ...                 field_id=518,
+            ...                 name="contains_nan",
+            ...                 field_type=BooleanType(),
+            ...                 is_optional=True,
+            ...                 doc="True if any file has a nan partition value",
+            ...             ),
+            ...         ),
+            ...         is_optional=True,
+            ...         doc="Summary for each partition",
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            field:
+
+        Returns:
+
+        """
+        field_id = field["field-id"]

Review Comment:
   Good one. I was thinking as a potential follow-up is setting id's just sequentially when traversing the schema if the id's aren't missing. This should be a conscious choice then 👍🏻 



-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
Fokko commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r877583548


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,274 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Util class for converting between Avro and Iceberg schemas
+
+"""
+import logging
+from typing import Any, Dict, List, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DoubleType,
+    FloatType,
+    IntegerType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+)
+
+logger = logging.getLogger(__name__)
+
+
+class AvroSchemaConversion:
+    PRIMITIVE_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "boolean": BooleanType(),
+        "bytes": BinaryType(),
+        "date": DateType(),
+        "double": DoubleType(),
+        "float": FloatType(),
+        "int": IntegerType(),
+        "long": LongType(),
+        "string": StringType(),
+        "time-millis": TimeType(),
+        "timestamp-millis": TimestampType(),
+    }
+
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expect to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into a Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+
+        Todo:
+            * Implement full support for unions
+            * Implement logical types
+        """
+        fields = self._parse_record(avro_schema)
+        return Schema(*fields.fields, schema_id=1)  # type: ignore
+
+    def _parse_record(self, avro_field: Dict[str, Any]) -> StructType:
+        return StructType(*[self._parse_field(field) for field in avro_field["fields"]])
+
+    def _resolve_union(self, type_union: Union[Dict, List, str]) -> Tuple[Union[str, dict], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        """
+        avro_types: Union[Dict, List]
+        if isinstance(type_union, str):
+            avro_types = [type_union]
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        # Filter the null value, so we know the actual type
+        avro_types = list(filter(lambda t: t != "null", avro_types))
+
+        if len(avro_types) != 1:
+            raise ValueError("Support for unions is yet to be implemented")
+
+        avro_type = avro_types[0]
+
+        return avro_type, is_optional
+
+    def _parse_field(self, field: Dict[str, Any]) -> NestedField:
+        """
+        Recursively walks through the Schema, constructing the Iceberg schema
+
+        Examples:
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {
+            ...             "name": "partitions",
+            ...                "type": [
+            ...                "null",
+            ...                {
+            ...                        "type": "array",
+            ...                        "items": {
+            ...                            "type": "record",
+            ...                            "name": "r508",
+            ...                            "fields": [
+            ...                                {
+            ...                                    "name": "contains_null",
+            ...                                    "type": "boolean",
+            ...                                    "doc": "True if any file has a null partition value",
+            ...                                    "field-id": 509,
+            ...                                },
+            ...                                {
+            ...                                    "name": "contains_nan",
+            ...                                    "type": ["null", "boolean"],
+            ...                                    "doc": "True if any file has a nan partition value",
+            ...                                    "default": None,
+            ...                                    "field-id": 518,
+            ...                                },
+            ...                            ],
+            ...                        },
+            ...                        "element-id": 508,
+            ...                    },
+            ...                ],
+            ...                "doc": "Summary for each partition",
+            ...                "default": None,
+            ...                "field-id": 507,
+            ...            },
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=507,
+            ...         name="partitions",
+            ...         field_type=StructType(
+            ...             NestedField(
+            ...                 field_id=509,
+            ...                 name="contains_null",
+            ...                 field_type=BooleanType(),
+            ...                 is_optional=False,
+            ...                 doc="True if any file has a null partition value",
+            ...             ),
+            ...             NestedField(
+            ...                 field_id=518,
+            ...                 name="contains_nan",
+            ...                 field_type=BooleanType(),
+            ...                 is_optional=True,
+            ...                 doc="True if any file has a nan partition value",
+            ...             ),
+            ...         ),
+            ...         is_optional=True,
+            ...         doc="Summary for each partition",
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            field:
+
+        Returns:
+
+        """
+        field_id = field["field-id"]
+        field_name = field["name"]
+        field_doc = field.get("doc")
+
+        avro_type, is_optional = self._resolve_union(field["type"])
+        if isinstance(avro_type, dict):
+            avro_type = avro_type["type"]
+
+        if isinstance(avro_type, str) and avro_type in AvroSchemaConversion.PRIMITIVE_FIELD_TYPE_MAP:
+            return NestedField(
+                field_id=field_id,
+                name=field_name,
+                field_type=AvroSchemaConversion.PRIMITIVE_FIELD_TYPE_MAP[avro_type],
+                is_optional=is_optional,
+                doc=field_doc,
+            )
+        elif avro_type == "record":
+            return NestedField(
+                field_id=field_id, name=field_name, field_type=self._parse_record(field), is_optional=is_optional, doc=field_doc
+            )
+        elif avro_type == "array":
+            inner_type, _ = self._resolve_union(field["type"])
+            assert isinstance(inner_type, dict)
+            inner_type["items"]["field-id"] = inner_type["element-id"]
+            inner_field = self._parse_field(inner_type["items"])
+
+            return NestedField(
+                field_id=field_id, name=field_name, field_type=inner_field.type, is_optional=is_optional, doc=field_doc
+            )
+        elif avro_type == "map":
+            return NestedField(
+                field_id=field_id,
+                name=field_name,
+                field_type=MapType(
+                    key_id=field["key-id"],
+                    key_type=StringType(),
+                    value_id=field["value-id"],
+                    value_type=self._parse_field(field["values"]),

Review Comment:
   Fixed both of 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] Fokko commented on a diff in pull request #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
Fokko commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r884323646


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,451 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Utility class for converting between Avro and Iceberg schemas
+
+"""
+from __future__ import annotations
+
+import logging
+from typing import Any, Dict, List, Tuple
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IcebergType,
+    IntegerType,
+    ListType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+    UUIDType,
+)
+
+logger = logging.getLogger(__name__)
+
+PRIMITIVE_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "boolean": BooleanType(),
+    "bytes": BinaryType(),
+    "double": DoubleType(),
+    "float": FloatType(),
+    "int": IntegerType(),
+    "long": LongType(),
+    "string": StringType(),
+}
+
+LOGICAL_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "date": DateType(),
+    "time-millis": TimeType(),
+    "timestamp-millis": TimestampType(),
+    "uuid": UUIDType(),
+}
+
+
+class AvroSchemaConversion:
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expects to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into an Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+        """
+        return Schema(*[self._convert_field(field) for field in avro_schema["fields"]], schema_id=1)
+
+    def _resolve_union(self, type_union: Dict | List | str) -> Tuple[str | Dict[str, Any], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        Raises:
+            TypeError: In the case non-optional union types are encountered
+        """
+        avro_types: Dict | List
+        if isinstance(type_union, str):
+            # It is a primitive and required
+            return type_union, False
+        elif isinstance(type_union, dict):
+            # It is a context and required
+            return type_union, False
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        if len(avro_types) > 2:
+            raise TypeError("Non-optional types aren't part of the Iceberg specification")
+
+        # Filter the null value and return the type
+        return list(filter(lambda t: t != "null", avro_types))[0], is_optional
+
+    def _resolve_inner_type(self, avro_type: Dict[str, Any], inner_field_name: str, id_field: str) -> Tuple[IcebergType, bool]:
+        """
+        Removes any unions inside an array or map, and appends the field-id downstream (if required)
+
+        Args:
+            avro_type: The parent Avro type
+            inner_field_name: For example 'items' in case of an array
+            id_field: Contains the element or key/value id, for example element-id, to be copied to the field-id
+
+        Returns:
+            The inner type and the nullability
+        """
+        plain_type, element_is_optional = self._resolve_union(avro_type[inner_field_name])
+        if isinstance(plain_type, dict):
+            # We need the element-id downstream
+            plain_type["field-id"] = avro_type[id_field]
+        inner_field = self._convert_schema(plain_type)
+        return inner_field, element_is_optional
+
+    def _convert_schema(self, avro_type: str | Dict[str, Any]) -> IcebergType:
+        """
+        Resolves the Avro type
+
+        Args:
+            avro_type: The Avro type, can be simple or complex
+
+        Returns:
+            The equivalent IcebergType
+
+        Raises:
+            ValueError: When there are unknown types
+        """
+        if isinstance(avro_type, str):
+            return PRIMITIVE_FIELD_TYPE_MAPPING[avro_type]
+        elif isinstance(avro_type, dict):
+            if "logicalType" in avro_type:
+                return self._convert_logical_type(avro_type)
+            else:
+                type_identifier = avro_type["type"]
+                if type_identifier == "record":
+                    return self._convert_record_type(avro_type)
+                elif type_identifier == "array":
+                    return self._convert_array_type(avro_type)
+                elif type_identifier == "map":
+                    return self._convert_map_type(avro_type)
+                elif type_identifier == "fixed":
+                    return self._convert_fixed_type(avro_type)
+                else:
+                    raise ValueError(f"Unknown type: {avro_type}")
+        else:
+            raise ValueError(f"Unknown type: {avro_type}")
+
+    def _convert_field(self, field: Dict[str, Any]) -> NestedField:
+        """
+        Converts an Avro field into an Iceberg equivalent field
+        Args:
+            field: The Avro field
+
+        Returns:
+            The Iceberg equivalent field
+        """
+        assert "field-id" in field, "Missing field-id in the Avro field, this is required for converting it to an Iceberg schema"
+
+        plain_type, is_optional = self._resolve_union(field["type"])
+
+        return NestedField(
+            field_id=field["field-id"],
+            name=field["name"],
+            field_type=self._convert_schema(plain_type),
+            is_optional=is_optional,
+            doc=field.get("doc"),
+        )
+
+    def _convert_record_type(self, record_type: Dict[str, Any]) -> StructType:
+        """
+        Converts the fields from a record into an Iceberg struct
+
+        Examples:
+            >>> from iceberg.utils.schema_conversion import AvroSchemaConversion
+            >>> record_type = {
+            ...     "type": "record",
+            ...     "name": "r508",
+            ...     "fields": [{
+            ...         "name": "contains_null",
+            ...         "type": "boolean",
+            ...         "doc": "True if any file has a null partition value",
+            ...         "field-id": 509,
+            ...      }, {
+            ...          "name": "contains_nan",
+            ...          "type": ["null", "boolean"],
+            ...          "doc": "True if any file has a nan partition value",
+            ...          "default": None,
+            ...          "field-id": 518,
+            ...      }],
+            ... }
+            >>> actual = AvroSchemaConversion()._convert_record_type(record_type)
+            >>> expected = StructType(
+            ...     fields=(
+            ...         NestedField(
+            ...             field_id=509,
+            ...             name="contains_null",
+            ...             field_type=BooleanType(),
+            ...             is_optional=False,
+            ...             doc="True if any file has a null partition value",
+            ...         ),
+            ...         NestedField(
+            ...             field_id=518,
+            ...             name="contains_nan",
+            ...             field_type=BooleanType(),
+            ...             is_optional=True,
+            ...             doc="True if any file has a nan partition value",
+            ...         ),
+            ...     )
+            ... )
+            >>> expected == actual
+            True
+
+        Args:
+            record_type: The record type itself
+
+        Returns:
+        """
+        return StructType(*[self._convert_field(field) for field in record_type["fields"]])
+
+    def _convert_array_type(self, array_type: Dict[str, Any]) -> ListType:
+        element_type, element_is_optional = self._resolve_inner_type(array_type, "items", "element-id")
+        return ListType(element_id=array_type["element-id"], element_type=element_type, element_is_optional=element_is_optional)
+
+    def _convert_map_type(self, map_type: Dict[str, Any]) -> MapType:
+        """
+        Args:
+            map_type: The dict that describes the Avro map type
+
+        Examples:
+            >>> from iceberg.utils.schema_conversion import AvroSchemaConversion
+            >>> avro_field = {
+            ...     "type": "map",
+            ...     "values": ["long", "null"],
+            ...     "key-id": 101,
+            ...     "value-id": 102,
+            ... }
+            >>> actual = AvroSchemaConversion()._convert_map_type(avro_field)
+            >>> expected = MapType(
+            ...     key_id=101,
+            ...     key_type=StringType(),
+            ...     value_id=102,
+            ...     value_type=LongType(),
+            ...     value_is_optional=True
+            ... )
+            >>> actual == expected
+            True
+
+        Returns: A MapType
+        """
+        value_type, value_is_optional = self._resolve_union(map_type["values"])
+        return MapType(
+            key_id=map_type["key-id"],
+            # Avro only supports string keys
+            key_type=StringType(),
+            value_id=map_type["value-id"],
+            value_type=self._convert_schema(value_type),
+            value_is_optional=value_is_optional,
+        )
+
+    def _convert_logical_type(self, avro_logical_type: Dict[str, Any]) -> IcebergType:
+        """
+        When a logical type is found, we'll resolve it here. For the decimal and map
+        we need to fetch more keys from the dict, and for the simple ones we can just
+        look it up in the mapping.
+
+        Examples:
+            >>> from iceberg.utils.schema_conversion import AvroSchemaConversion
+            >>> avro_logical_type = {
+            ...     "type": "int",
+            ...     "logicalType": "date"
+            ... }
+            >>> actual = AvroSchemaConversion()._convert_logical_type(avro_logical_type)
+            >>> actual == DateType()
+            True
+
+        Args:
+            avro_logical_type: The logical type
+
+        Returns:
+            The converted logical type
+
+        Raises:
+            ValueError: When the logical type is unknown
+        """
+        logical_type = avro_logical_type["logicalType"]
+        if logical_type == "decimal":
+            return self._convert_logical_decimal_type(avro_logical_type)
+        elif logical_type == "map":
+            return self._convert_logical_map_type(avro_logical_type)
+        elif logical_type in LOGICAL_FIELD_TYPE_MAPPING:
+            return LOGICAL_FIELD_TYPE_MAPPING[logical_type]

Review Comment:
   We can. I've added logic to lookup the logical/physical combination 👍🏻 



-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r884298460


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,451 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Utility class for converting between Avro and Iceberg schemas
+
+"""
+from __future__ import annotations
+
+import logging
+from typing import Any, Dict, List, Tuple
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IcebergType,
+    IntegerType,
+    ListType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+    UUIDType,
+)
+
+logger = logging.getLogger(__name__)
+
+PRIMITIVE_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "boolean": BooleanType(),
+    "bytes": BinaryType(),
+    "double": DoubleType(),
+    "float": FloatType(),
+    "int": IntegerType(),
+    "long": LongType(),
+    "string": StringType(),
+}
+
+LOGICAL_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "date": DateType(),
+    "time-millis": TimeType(),
+    "timestamp-millis": TimestampType(),
+    "uuid": UUIDType(),
+}
+
+
+class AvroSchemaConversion:
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expects to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into an Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+        """
+        return Schema(*[self._convert_field(field) for field in avro_schema["fields"]], schema_id=1)
+
+    def _resolve_union(self, type_union: Dict | List | str) -> Tuple[str | Dict[str, Any], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        Raises:
+            TypeError: In the case non-optional union types are encountered
+        """
+        avro_types: Dict | List
+        if isinstance(type_union, str):
+            # It is a primitive and required
+            return type_union, False
+        elif isinstance(type_union, dict):
+            # It is a context and required
+            return type_union, False
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        if len(avro_types) > 2:
+            raise TypeError("Non-optional types aren't part of the Iceberg specification")
+
+        # Filter the null value and return the type
+        return list(filter(lambda t: t != "null", avro_types))[0], is_optional
+
+    def _resolve_inner_type(self, avro_type: Dict[str, Any], inner_field_name: str, id_field: str) -> Tuple[IcebergType, bool]:
+        """
+        Removes any unions inside an array or map, and appends the field-id downstream (if required)
+
+        Args:
+            avro_type: The parent Avro type
+            inner_field_name: For example 'items' in case of an array
+            id_field: Contains the element or key/value id, for example element-id, to be copied to the field-id
+
+        Returns:
+            The inner type and the nullability
+        """
+        plain_type, element_is_optional = self._resolve_union(avro_type[inner_field_name])
+        if isinstance(plain_type, dict):
+            # We need the element-id downstream
+            plain_type["field-id"] = avro_type[id_field]
+        inner_field = self._convert_schema(plain_type)
+        return inner_field, element_is_optional
+
+    def _convert_schema(self, avro_type: str | Dict[str, Any]) -> IcebergType:
+        """
+        Resolves the Avro type
+
+        Args:
+            avro_type: The Avro type, can be simple or complex
+
+        Returns:
+            The equivalent IcebergType
+
+        Raises:
+            ValueError: When there are unknown types
+        """
+        if isinstance(avro_type, str):
+            return PRIMITIVE_FIELD_TYPE_MAPPING[avro_type]
+        elif isinstance(avro_type, dict):
+            if "logicalType" in avro_type:
+                return self._convert_logical_type(avro_type)
+            else:
+                type_identifier = avro_type["type"]
+                if type_identifier == "record":
+                    return self._convert_record_type(avro_type)
+                elif type_identifier == "array":
+                    return self._convert_array_type(avro_type)
+                elif type_identifier == "map":
+                    return self._convert_map_type(avro_type)
+                elif type_identifier == "fixed":
+                    return self._convert_fixed_type(avro_type)
+                else:
+                    raise ValueError(f"Unknown type: {avro_type}")
+        else:
+            raise ValueError(f"Unknown type: {avro_type}")
+
+    def _convert_field(self, field: Dict[str, Any]) -> NestedField:
+        """
+        Converts an Avro field into an Iceberg equivalent field
+        Args:
+            field: The Avro field
+
+        Returns:
+            The Iceberg equivalent field
+        """
+        assert "field-id" in field, "Missing field-id in the Avro field, this is required for converting it to an Iceberg schema"
+
+        plain_type, is_optional = self._resolve_union(field["type"])
+
+        return NestedField(
+            field_id=field["field-id"],
+            name=field["name"],
+            field_type=self._convert_schema(plain_type),
+            is_optional=is_optional,
+            doc=field.get("doc"),
+        )
+
+    def _convert_record_type(self, record_type: Dict[str, Any]) -> StructType:
+        """
+        Converts the fields from a record into an Iceberg struct
+
+        Examples:
+            >>> from iceberg.utils.schema_conversion import AvroSchemaConversion
+            >>> record_type = {
+            ...     "type": "record",
+            ...     "name": "r508",
+            ...     "fields": [{
+            ...         "name": "contains_null",
+            ...         "type": "boolean",
+            ...         "doc": "True if any file has a null partition value",
+            ...         "field-id": 509,
+            ...      }, {
+            ...          "name": "contains_nan",
+            ...          "type": ["null", "boolean"],
+            ...          "doc": "True if any file has a nan partition value",
+            ...          "default": None,
+            ...          "field-id": 518,
+            ...      }],
+            ... }
+            >>> actual = AvroSchemaConversion()._convert_record_type(record_type)
+            >>> expected = StructType(
+            ...     fields=(
+            ...         NestedField(
+            ...             field_id=509,
+            ...             name="contains_null",
+            ...             field_type=BooleanType(),
+            ...             is_optional=False,
+            ...             doc="True if any file has a null partition value",
+            ...         ),
+            ...         NestedField(
+            ...             field_id=518,
+            ...             name="contains_nan",
+            ...             field_type=BooleanType(),
+            ...             is_optional=True,
+            ...             doc="True if any file has a nan partition value",
+            ...         ),
+            ...     )
+            ... )
+            >>> expected == actual
+            True
+
+        Args:
+            record_type: The record type itself
+
+        Returns:
+        """
+        return StructType(*[self._convert_field(field) for field in record_type["fields"]])
+
+    def _convert_array_type(self, array_type: Dict[str, Any]) -> ListType:
+        element_type, element_is_optional = self._resolve_inner_type(array_type, "items", "element-id")
+        return ListType(element_id=array_type["element-id"], element_type=element_type, element_is_optional=element_is_optional)
+
+    def _convert_map_type(self, map_type: Dict[str, Any]) -> MapType:
+        """
+        Args:
+            map_type: The dict that describes the Avro map type
+
+        Examples:
+            >>> from iceberg.utils.schema_conversion import AvroSchemaConversion
+            >>> avro_field = {
+            ...     "type": "map",
+            ...     "values": ["long", "null"],
+            ...     "key-id": 101,
+            ...     "value-id": 102,
+            ... }
+            >>> actual = AvroSchemaConversion()._convert_map_type(avro_field)
+            >>> expected = MapType(
+            ...     key_id=101,
+            ...     key_type=StringType(),
+            ...     value_id=102,
+            ...     value_type=LongType(),
+            ...     value_is_optional=True
+            ... )
+            >>> actual == expected
+            True
+
+        Returns: A MapType
+        """
+        value_type, value_is_optional = self._resolve_union(map_type["values"])
+        return MapType(
+            key_id=map_type["key-id"],
+            # Avro only supports string keys
+            key_type=StringType(),
+            value_id=map_type["value-id"],
+            value_type=self._convert_schema(value_type),
+            value_is_optional=value_is_optional,
+        )
+
+    def _convert_logical_type(self, avro_logical_type: Dict[str, Any]) -> IcebergType:
+        """
+        When a logical type is found, we'll resolve it here. For the decimal and map
+        we need to fetch more keys from the dict, and for the simple ones we can just
+        look it up in the mapping.
+
+        Examples:
+            >>> from iceberg.utils.schema_conversion import AvroSchemaConversion
+            >>> avro_logical_type = {
+            ...     "type": "int",
+            ...     "logicalType": "date"
+            ... }
+            >>> actual = AvroSchemaConversion()._convert_logical_type(avro_logical_type)
+            >>> actual == DateType()
+            True
+
+        Args:
+            avro_logical_type: The logical type
+
+        Returns:
+            The converted logical type
+
+        Raises:
+            ValueError: When the logical type is unknown
+        """
+        logical_type = avro_logical_type["logicalType"]
+        if logical_type == "decimal":
+            return self._convert_logical_decimal_type(avro_logical_type)
+        elif logical_type == "map":
+            return self._convert_logical_map_type(avro_logical_type)
+        elif logical_type in LOGICAL_FIELD_TYPE_MAPPING:
+            return LOGICAL_FIELD_TYPE_MAPPING[logical_type]
+        else:
+            raise ValueError(f"Unknown logical type: {avro_logical_type}")
+
+    def _convert_logical_decimal_type(self, avro_type: Dict[str, Any]) -> DecimalType:
+        """
+        Args:
+            avro_type: The Avro type
+
+        Examples:
+            >>> from iceberg.utils.schema_conversion import AvroSchemaConversion
+            >>> avro_decimal_type = {
+            ...     "type": "bytes",
+            ...     "logicalType": "decimal",
+            ...     "precision": 19,
+            ...     "scale": 25
+            ... }
+            >>> actual = AvroSchemaConversion()._convert_logical_decimal_type(avro_decimal_type)
+            >>> expected = DecimalType(
+            ...     precision=19,
+            ...     scale=25
+            ... )
+            >>> actual == expected
+            True
+
+        Returns:
+            A Iceberg DecimalType
+        """
+        return DecimalType(precision=avro_type["precision"], scale=avro_type["scale"])
+
+    def _convert_logical_map_type(self, avro_type: Dict[str, Any]) -> MapType:
+        """
+        In the case where a map hasn't a key as a type you can use a logical map to
+        still encode this in Avro
+
+        Args:
+            avro_type: The Avro Type
+
+        Examples:
+            >>> from iceberg.utils.schema_conversion import AvroSchemaConversion
+            >>> avro_type = {
+            ...     "type": "array",
+            ...     "logicalType": "map",
+            ...     "items": {
+            ...         "type": "record",
+            ...         "name": "k101_v102",
+            ...         "fields": [
+            ...             {"name": "key", "type": "int", "field-id": 101},
+            ...             {"name": "value", "type": "string", "field-id": 102},
+            ...         ],
+            ...     },
+            ... }
+            >>> actual = AvroSchemaConversion()._convert_logical_map_type(avro_type)
+            >>> expected = MapType(
+            ...         key_id=101,
+            ...         key_type=IntegerType(),
+            ...         value_id=102,
+            ...         value_type=StringType(),
+            ...         value_is_optional=False
+            ... )
+            >>> actual == expected
+            True
+
+        .. _Apache Iceberg specification:
+            https://iceberg.apache.org/spec/#appendix-a-format-specific-requirements
+
+        Returns:
+            The logical map
+        """
+        fields = avro_type["items"]["fields"]
+        assert len(fields) == 2, f"Expected two fields in the logical map, but got: {fields}"
+        key = self._convert_field(fields[0])
+        value = self._convert_field(fields[1])

Review Comment:
   This is probably okay, but I don't think that there's a guarantee that the fields are in order. It would be good to check the field names.



-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r884297946


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,451 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Utility class for converting between Avro and Iceberg schemas
+
+"""
+from __future__ import annotations
+
+import logging
+from typing import Any, Dict, List, Tuple
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IcebergType,
+    IntegerType,
+    ListType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+    UUIDType,
+)
+
+logger = logging.getLogger(__name__)
+
+PRIMITIVE_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "boolean": BooleanType(),
+    "bytes": BinaryType(),
+    "double": DoubleType(),
+    "float": FloatType(),
+    "int": IntegerType(),
+    "long": LongType(),
+    "string": StringType(),
+}
+
+LOGICAL_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "date": DateType(),
+    "time-millis": TimeType(),
+    "timestamp-millis": TimestampType(),
+    "uuid": UUIDType(),
+}
+
+
+class AvroSchemaConversion:
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expects to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into an Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+        """
+        return Schema(*[self._convert_field(field) for field in avro_schema["fields"]], schema_id=1)
+
+    def _resolve_union(self, type_union: Dict | List | str) -> Tuple[str | Dict[str, Any], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        Raises:
+            TypeError: In the case non-optional union types are encountered
+        """
+        avro_types: Dict | List
+        if isinstance(type_union, str):
+            # It is a primitive and required
+            return type_union, False
+        elif isinstance(type_union, dict):
+            # It is a context and required
+            return type_union, False
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        if len(avro_types) > 2:
+            raise TypeError("Non-optional types aren't part of the Iceberg specification")
+
+        # Filter the null value and return the type
+        return list(filter(lambda t: t != "null", avro_types))[0], is_optional
+
+    def _resolve_inner_type(self, avro_type: Dict[str, Any], inner_field_name: str, id_field: str) -> Tuple[IcebergType, bool]:
+        """
+        Removes any unions inside an array or map, and appends the field-id downstream (if required)
+
+        Args:
+            avro_type: The parent Avro type
+            inner_field_name: For example 'items' in case of an array
+            id_field: Contains the element or key/value id, for example element-id, to be copied to the field-id
+
+        Returns:
+            The inner type and the nullability
+        """
+        plain_type, element_is_optional = self._resolve_union(avro_type[inner_field_name])
+        if isinstance(plain_type, dict):
+            # We need the element-id downstream
+            plain_type["field-id"] = avro_type[id_field]
+        inner_field = self._convert_schema(plain_type)
+        return inner_field, element_is_optional
+
+    def _convert_schema(self, avro_type: str | Dict[str, Any]) -> IcebergType:
+        """
+        Resolves the Avro type
+
+        Args:
+            avro_type: The Avro type, can be simple or complex
+
+        Returns:
+            The equivalent IcebergType
+
+        Raises:
+            ValueError: When there are unknown types
+        """
+        if isinstance(avro_type, str):
+            return PRIMITIVE_FIELD_TYPE_MAPPING[avro_type]
+        elif isinstance(avro_type, dict):
+            if "logicalType" in avro_type:
+                return self._convert_logical_type(avro_type)
+            else:
+                type_identifier = avro_type["type"]
+                if type_identifier == "record":
+                    return self._convert_record_type(avro_type)
+                elif type_identifier == "array":
+                    return self._convert_array_type(avro_type)
+                elif type_identifier == "map":
+                    return self._convert_map_type(avro_type)
+                elif type_identifier == "fixed":
+                    return self._convert_fixed_type(avro_type)
+                else:
+                    raise ValueError(f"Unknown type: {avro_type}")
+        else:
+            raise ValueError(f"Unknown type: {avro_type}")
+
+    def _convert_field(self, field: Dict[str, Any]) -> NestedField:
+        """
+        Converts an Avro field into an Iceberg equivalent field
+        Args:
+            field: The Avro field
+
+        Returns:
+            The Iceberg equivalent field
+        """
+        assert "field-id" in field, "Missing field-id in the Avro field, this is required for converting it to an Iceberg schema"
+
+        plain_type, is_optional = self._resolve_union(field["type"])
+
+        return NestedField(
+            field_id=field["field-id"],
+            name=field["name"],
+            field_type=self._convert_schema(plain_type),
+            is_optional=is_optional,
+            doc=field.get("doc"),
+        )
+
+    def _convert_record_type(self, record_type: Dict[str, Any]) -> StructType:
+        """
+        Converts the fields from a record into an Iceberg struct
+
+        Examples:
+            >>> from iceberg.utils.schema_conversion import AvroSchemaConversion
+            >>> record_type = {
+            ...     "type": "record",
+            ...     "name": "r508",
+            ...     "fields": [{
+            ...         "name": "contains_null",
+            ...         "type": "boolean",
+            ...         "doc": "True if any file has a null partition value",
+            ...         "field-id": 509,
+            ...      }, {
+            ...          "name": "contains_nan",
+            ...          "type": ["null", "boolean"],
+            ...          "doc": "True if any file has a nan partition value",
+            ...          "default": None,
+            ...          "field-id": 518,
+            ...      }],
+            ... }
+            >>> actual = AvroSchemaConversion()._convert_record_type(record_type)
+            >>> expected = StructType(
+            ...     fields=(
+            ...         NestedField(
+            ...             field_id=509,
+            ...             name="contains_null",
+            ...             field_type=BooleanType(),
+            ...             is_optional=False,
+            ...             doc="True if any file has a null partition value",
+            ...         ),
+            ...         NestedField(
+            ...             field_id=518,
+            ...             name="contains_nan",
+            ...             field_type=BooleanType(),
+            ...             is_optional=True,
+            ...             doc="True if any file has a nan partition value",
+            ...         ),
+            ...     )
+            ... )
+            >>> expected == actual
+            True
+
+        Args:
+            record_type: The record type itself
+
+        Returns:
+        """
+        return StructType(*[self._convert_field(field) for field in record_type["fields"]])
+
+    def _convert_array_type(self, array_type: Dict[str, Any]) -> ListType:
+        element_type, element_is_optional = self._resolve_inner_type(array_type, "items", "element-id")
+        return ListType(element_id=array_type["element-id"], element_type=element_type, element_is_optional=element_is_optional)
+
+    def _convert_map_type(self, map_type: Dict[str, Any]) -> MapType:
+        """
+        Args:
+            map_type: The dict that describes the Avro map type
+
+        Examples:
+            >>> from iceberg.utils.schema_conversion import AvroSchemaConversion
+            >>> avro_field = {
+            ...     "type": "map",
+            ...     "values": ["long", "null"],
+            ...     "key-id": 101,
+            ...     "value-id": 102,
+            ... }
+            >>> actual = AvroSchemaConversion()._convert_map_type(avro_field)
+            >>> expected = MapType(
+            ...     key_id=101,
+            ...     key_type=StringType(),
+            ...     value_id=102,
+            ...     value_type=LongType(),
+            ...     value_is_optional=True
+            ... )
+            >>> actual == expected
+            True
+
+        Returns: A MapType
+        """
+        value_type, value_is_optional = self._resolve_union(map_type["values"])
+        return MapType(
+            key_id=map_type["key-id"],
+            # Avro only supports string keys
+            key_type=StringType(),
+            value_id=map_type["value-id"],
+            value_type=self._convert_schema(value_type),
+            value_is_optional=value_is_optional,
+        )
+
+    def _convert_logical_type(self, avro_logical_type: Dict[str, Any]) -> IcebergType:
+        """
+        When a logical type is found, we'll resolve it here. For the decimal and map
+        we need to fetch more keys from the dict, and for the simple ones we can just
+        look it up in the mapping.
+
+        Examples:
+            >>> from iceberg.utils.schema_conversion import AvroSchemaConversion
+            >>> avro_logical_type = {
+            ...     "type": "int",
+            ...     "logicalType": "date"
+            ... }
+            >>> actual = AvroSchemaConversion()._convert_logical_type(avro_logical_type)
+            >>> actual == DateType()
+            True
+
+        Args:
+            avro_logical_type: The logical type
+
+        Returns:
+            The converted logical type
+
+        Raises:
+            ValueError: When the logical type is unknown
+        """
+        logical_type = avro_logical_type["logicalType"]
+        if logical_type == "decimal":
+            return self._convert_logical_decimal_type(avro_logical_type)
+        elif logical_type == "map":
+            return self._convert_logical_map_type(avro_logical_type)
+        elif logical_type in LOGICAL_FIELD_TYPE_MAPPING:
+            return LOGICAL_FIELD_TYPE_MAPPING[logical_type]

Review Comment:
   Should we also validate the underlying type? The Java library does this automatically, but I don't think there's any validation in python that the actual type for a `date` is `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] rdblue commented on a diff in pull request #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r884297208


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,451 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Utility class for converting between Avro and Iceberg schemas
+
+"""
+from __future__ import annotations
+
+import logging
+from typing import Any, Dict, List, Tuple
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IcebergType,
+    IntegerType,
+    ListType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+    UUIDType,
+)
+
+logger = logging.getLogger(__name__)
+
+PRIMITIVE_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "boolean": BooleanType(),
+    "bytes": BinaryType(),
+    "double": DoubleType(),
+    "float": FloatType(),
+    "int": IntegerType(),
+    "long": LongType(),
+    "string": StringType(),
+}
+
+LOGICAL_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "date": DateType(),
+    "time-millis": TimeType(),
+    "timestamp-millis": TimestampType(),
+    "uuid": UUIDType(),
+}
+
+
+class AvroSchemaConversion:
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expects to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into an Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+        """
+        return Schema(*[self._convert_field(field) for field in avro_schema["fields"]], schema_id=1)
+
+    def _resolve_union(self, type_union: Dict | List | str) -> Tuple[str | Dict[str, Any], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        Raises:
+            TypeError: In the case non-optional union types are encountered
+        """
+        avro_types: Dict | List
+        if isinstance(type_union, str):
+            # It is a primitive and required
+            return type_union, False
+        elif isinstance(type_union, dict):
+            # It is a context and required
+            return type_union, False
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        if len(avro_types) > 2:
+            raise TypeError("Non-optional types aren't part of the Iceberg specification")
+
+        # Filter the null value and return the type
+        return list(filter(lambda t: t != "null", avro_types))[0], is_optional
+
+    def _resolve_inner_type(self, avro_type: Dict[str, Any], inner_field_name: str, id_field: str) -> Tuple[IcebergType, bool]:
+        """
+        Removes any unions inside an array or map, and appends the field-id downstream (if required)
+
+        Args:
+            avro_type: The parent Avro type
+            inner_field_name: For example 'items' in case of an array
+            id_field: Contains the element or key/value id, for example element-id, to be copied to the field-id
+
+        Returns:
+            The inner type and the nullability
+        """
+        plain_type, element_is_optional = self._resolve_union(avro_type[inner_field_name])
+        if isinstance(plain_type, dict):
+            # We need the element-id downstream
+            plain_type["field-id"] = avro_type[id_field]
+        inner_field = self._convert_schema(plain_type)
+        return inner_field, element_is_optional
+
+    def _convert_schema(self, avro_type: str | Dict[str, Any]) -> IcebergType:
+        """
+        Resolves the Avro type
+
+        Args:
+            avro_type: The Avro type, can be simple or complex
+
+        Returns:
+            The equivalent IcebergType
+
+        Raises:
+            ValueError: When there are unknown types
+        """
+        if isinstance(avro_type, str):
+            return PRIMITIVE_FIELD_TYPE_MAPPING[avro_type]
+        elif isinstance(avro_type, dict):
+            if "logicalType" in avro_type:
+                return self._convert_logical_type(avro_type)
+            else:
+                type_identifier = avro_type["type"]
+                if type_identifier == "record":
+                    return self._convert_record_type(avro_type)
+                elif type_identifier == "array":
+                    return self._convert_array_type(avro_type)
+                elif type_identifier == "map":
+                    return self._convert_map_type(avro_type)
+                elif type_identifier == "fixed":
+                    return self._convert_fixed_type(avro_type)
+                else:
+                    raise ValueError(f"Unknown type: {avro_type}")
+        else:
+            raise ValueError(f"Unknown type: {avro_type}")
+
+    def _convert_field(self, field: Dict[str, Any]) -> NestedField:
+        """
+        Converts an Avro field into an Iceberg equivalent field
+        Args:
+            field: The Avro field
+
+        Returns:
+            The Iceberg equivalent field
+        """
+        assert "field-id" in field, "Missing field-id in the Avro field, this is required for converting it to an Iceberg schema"
+
+        plain_type, is_optional = self._resolve_union(field["type"])
+
+        return NestedField(
+            field_id=field["field-id"],
+            name=field["name"],
+            field_type=self._convert_schema(plain_type),
+            is_optional=is_optional,
+            doc=field.get("doc"),
+        )
+
+    def _convert_record_type(self, record_type: Dict[str, Any]) -> StructType:
+        """
+        Converts the fields from a record into an Iceberg struct
+
+        Examples:
+            >>> from iceberg.utils.schema_conversion import AvroSchemaConversion
+            >>> record_type = {
+            ...     "type": "record",
+            ...     "name": "r508",
+            ...     "fields": [{
+            ...         "name": "contains_null",
+            ...         "type": "boolean",
+            ...         "doc": "True if any file has a null partition value",
+            ...         "field-id": 509,
+            ...      }, {
+            ...          "name": "contains_nan",
+            ...          "type": ["null", "boolean"],
+            ...          "doc": "True if any file has a nan partition value",
+            ...          "default": None,
+            ...          "field-id": 518,
+            ...      }],
+            ... }
+            >>> actual = AvroSchemaConversion()._convert_record_type(record_type)
+            >>> expected = StructType(
+            ...     fields=(
+            ...         NestedField(
+            ...             field_id=509,
+            ...             name="contains_null",
+            ...             field_type=BooleanType(),
+            ...             is_optional=False,
+            ...             doc="True if any file has a null partition value",
+            ...         ),
+            ...         NestedField(
+            ...             field_id=518,
+            ...             name="contains_nan",
+            ...             field_type=BooleanType(),
+            ...             is_optional=True,
+            ...             doc="True if any file has a nan partition value",
+            ...         ),
+            ...     )
+            ... )
+            >>> expected == actual
+            True
+
+        Args:
+            record_type: The record type itself
+
+        Returns:
+        """
+        return StructType(*[self._convert_field(field) for field in record_type["fields"]])
+
+    def _convert_array_type(self, array_type: Dict[str, Any]) -> ListType:
+        element_type, element_is_optional = self._resolve_inner_type(array_type, "items", "element-id")
+        return ListType(element_id=array_type["element-id"], element_type=element_type, element_is_optional=element_is_optional)

Review Comment:
   `_convert_field` uses an assertion to ensure that `field-id` is present. Should this do the same?



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r884298784


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,451 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Utility class for converting between Avro and Iceberg schemas
+
+"""
+from __future__ import annotations
+
+import logging
+from typing import Any, Dict, List, Tuple
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IcebergType,
+    IntegerType,
+    ListType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+    UUIDType,
+)
+
+logger = logging.getLogger(__name__)
+
+PRIMITIVE_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "boolean": BooleanType(),
+    "bytes": BinaryType(),
+    "double": DoubleType(),
+    "float": FloatType(),
+    "int": IntegerType(),
+    "long": LongType(),
+    "string": StringType(),

Review Comment:
   `"enum"` is also converted to `StringType()`.



-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
Fokko commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r884315564


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,451 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Utility class for converting between Avro and Iceberg schemas
+
+"""
+from __future__ import annotations
+
+import logging
+from typing import Any, Dict, List, Tuple
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IcebergType,
+    IntegerType,
+    ListType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+    UUIDType,
+)
+
+logger = logging.getLogger(__name__)
+
+PRIMITIVE_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "boolean": BooleanType(),
+    "bytes": BinaryType(),
+    "double": DoubleType(),
+    "float": FloatType(),
+    "int": IntegerType(),
+    "long": LongType(),
+    "string": StringType(),
+}
+
+LOGICAL_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = {
+    "date": DateType(),
+    "time-millis": TimeType(),
+    "timestamp-millis": TimestampType(),
+    "uuid": UUIDType(),
+}
+
+
+class AvroSchemaConversion:
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expects to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into an Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+        """
+        return Schema(*[self._convert_field(field) for field in avro_schema["fields"]], schema_id=1)
+
+    def _resolve_union(self, type_union: Dict | List | str) -> Tuple[str | Dict[str, Any], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        Raises:
+            TypeError: In the case non-optional union types are encountered
+        """
+        avro_types: Dict | List
+        if isinstance(type_union, str):
+            # It is a primitive and required
+            return type_union, False
+        elif isinstance(type_union, dict):
+            # It is a context and required
+            return type_union, False
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        if len(avro_types) > 2:
+            raise TypeError("Non-optional types aren't part of the Iceberg specification")
+
+        # Filter the null value and return the type
+        return list(filter(lambda t: t != "null", avro_types))[0], is_optional
+
+    def _resolve_inner_type(self, avro_type: Dict[str, Any], inner_field_name: str, id_field: str) -> Tuple[IcebergType, bool]:
+        """
+        Removes any unions inside an array or map, and appends the field-id downstream (if required)
+
+        Args:
+            avro_type: The parent Avro type
+            inner_field_name: For example 'items' in case of an array
+            id_field: Contains the element or key/value id, for example element-id, to be copied to the field-id
+
+        Returns:
+            The inner type and the nullability
+        """
+        plain_type, element_is_optional = self._resolve_union(avro_type[inner_field_name])
+        if isinstance(plain_type, dict):
+            # We need the element-id downstream
+            plain_type["field-id"] = avro_type[id_field]
+        inner_field = self._convert_schema(plain_type)
+        return inner_field, element_is_optional
+
+    def _convert_schema(self, avro_type: str | Dict[str, Any]) -> IcebergType:
+        """
+        Resolves the Avro type
+
+        Args:
+            avro_type: The Avro type, can be simple or complex
+
+        Returns:
+            The equivalent IcebergType
+
+        Raises:
+            ValueError: When there are unknown types
+        """
+        if isinstance(avro_type, str):
+            return PRIMITIVE_FIELD_TYPE_MAPPING[avro_type]
+        elif isinstance(avro_type, dict):
+            if "logicalType" in avro_type:
+                return self._convert_logical_type(avro_type)
+            else:
+                type_identifier = avro_type["type"]
+                if type_identifier == "record":
+                    return self._convert_record_type(avro_type)
+                elif type_identifier == "array":
+                    return self._convert_array_type(avro_type)
+                elif type_identifier == "map":
+                    return self._convert_map_type(avro_type)
+                elif type_identifier == "fixed":
+                    return self._convert_fixed_type(avro_type)
+                else:
+                    raise ValueError(f"Unknown type: {avro_type}")
+        else:
+            raise ValueError(f"Unknown type: {avro_type}")
+
+    def _convert_field(self, field: Dict[str, Any]) -> NestedField:
+        """
+        Converts an Avro field into an Iceberg equivalent field
+        Args:
+            field: The Avro field
+
+        Returns:
+            The Iceberg equivalent field
+        """
+        assert "field-id" in field, "Missing field-id in the Avro field, this is required for converting it to an Iceberg schema"
+
+        plain_type, is_optional = self._resolve_union(field["type"])
+
+        return NestedField(
+            field_id=field["field-id"],
+            name=field["name"],
+            field_type=self._convert_schema(plain_type),
+            is_optional=is_optional,
+            doc=field.get("doc"),
+        )
+
+    def _convert_record_type(self, record_type: Dict[str, Any]) -> StructType:
+        """
+        Converts the fields from a record into an Iceberg struct
+
+        Examples:
+            >>> from iceberg.utils.schema_conversion import AvroSchemaConversion
+            >>> record_type = {
+            ...     "type": "record",
+            ...     "name": "r508",
+            ...     "fields": [{
+            ...         "name": "contains_null",
+            ...         "type": "boolean",
+            ...         "doc": "True if any file has a null partition value",
+            ...         "field-id": 509,
+            ...      }, {
+            ...          "name": "contains_nan",
+            ...          "type": ["null", "boolean"],
+            ...          "doc": "True if any file has a nan partition value",
+            ...          "default": None,
+            ...          "field-id": 518,
+            ...      }],
+            ... }
+            >>> actual = AvroSchemaConversion()._convert_record_type(record_type)
+            >>> expected = StructType(
+            ...     fields=(
+            ...         NestedField(
+            ...             field_id=509,
+            ...             name="contains_null",
+            ...             field_type=BooleanType(),
+            ...             is_optional=False,
+            ...             doc="True if any file has a null partition value",
+            ...         ),
+            ...         NestedField(
+            ...             field_id=518,
+            ...             name="contains_nan",
+            ...             field_type=BooleanType(),
+            ...             is_optional=True,
+            ...             doc="True if any file has a nan partition value",
+            ...         ),
+            ...     )
+            ... )
+            >>> expected == actual
+            True
+
+        Args:
+            record_type: The record type itself
+
+        Returns:
+        """
+        return StructType(*[self._convert_field(field) for field in record_type["fields"]])
+
+    def _convert_array_type(self, array_type: Dict[str, Any]) -> ListType:
+        element_type, element_is_optional = self._resolve_inner_type(array_type, "items", "element-id")
+        return ListType(element_id=array_type["element-id"], element_type=element_type, element_is_optional=element_is_optional)

Review Comment:
   Seems fair, I've added the check 👍🏻 



-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
Fokko commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r884314787


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,371 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Utility class for converting between Avro and Iceberg schemas
+
+"""
+import logging
+from typing import Any, Dict, List, Optional, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IcebergType,
+    IntegerType,
+    ListType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+    UUIDType,
+)
+
+logger = logging.getLogger(__name__)
+
+
+class AvroSchemaConversion:
+    PRIMITIVE_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "boolean": BooleanType(),
+        "bytes": BinaryType(),
+        "double": DoubleType(),
+        "float": FloatType(),
+        "int": IntegerType(),
+        "long": LongType(),
+        "string": StringType(),
+    }
+
+    LOGICAL_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "date": DateType(),
+        "time-millis": TimeType(),
+        "timestamp-millis": TimestampType(),
+        "uuid": UUIDType(),
+    }
+
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expects to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into a Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+        """
+        fields = self._parse_record(avro_schema)
+        return Schema(*fields.fields, schema_id=1)
+
+    def _parse_record(self, avro_field: Dict[str, Any]) -> StructType:
+        fields = [self._parse_field(field) for field in avro_field["fields"]]

Review Comment:
   Good one, I've added a check



-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r883249007


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,371 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Utility class for converting between Avro and Iceberg schemas
+
+"""
+import logging
+from typing import Any, Dict, List, Optional, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IcebergType,
+    IntegerType,
+    ListType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+    UUIDType,
+)
+
+logger = logging.getLogger(__name__)
+
+
+class AvroSchemaConversion:
+    PRIMITIVE_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "boolean": BooleanType(),
+        "bytes": BinaryType(),
+        "double": DoubleType(),
+        "float": FloatType(),
+        "int": IntegerType(),
+        "long": LongType(),
+        "string": StringType(),
+    }
+
+    LOGICAL_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "date": DateType(),
+        "time-millis": TimeType(),
+        "timestamp-millis": TimestampType(),
+        "uuid": UUIDType(),
+    }
+
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expects to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into a Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+        """
+        fields = self._parse_record(avro_schema)
+        return Schema(*fields.fields, schema_id=1)
+
+    def _parse_record(self, avro_field: Dict[str, Any]) -> StructType:
+        fields = [self._parse_field(field) for field in avro_field["fields"]]
+        return StructType(*fields)  # type: ignore
+
+    def _resolve_union(self, type_union: Union[Dict, List, str]) -> Tuple[Union[str, Dict[str, Any]], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        Raises:
+            TypeError: In the case non-optional union types are encountered
+        """
+        avro_types: Union[Dict, List]
+        if isinstance(type_union, str):
+            # It is a primitive and required
+            return type_union, False
+        elif isinstance(type_union, dict):
+            # It is a context and required
+            return type_union, False
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types

Review Comment:
   You could also compare the size of `avro_types` after filtering with `avro_types` before filtering.



-- 
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 #4742: Python: Convert Avro to Iceberg schema

Posted by GitBox <gi...@apache.org>.
Fokko commented on code in PR #4742:
URL: https://github.com/apache/iceberg/pull/4742#discussion_r883980032


##########
python/src/iceberg/utils/schema_conversion.py:
##########
@@ -0,0 +1,371 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Utility class for converting between Avro and Iceberg schemas
+
+"""
+import logging
+from typing import Any, Dict, List, Optional, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.types import (
+    BinaryType,
+    BooleanType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FixedType,
+    FloatType,
+    IcebergType,
+    IntegerType,
+    ListType,
+    LongType,
+    MapType,
+    NestedField,
+    PrimitiveType,
+    StringType,
+    StructType,
+    TimestampType,
+    TimeType,
+    UUIDType,
+)
+
+logger = logging.getLogger(__name__)
+
+
+class AvroSchemaConversion:
+    PRIMITIVE_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "boolean": BooleanType(),
+        "bytes": BinaryType(),
+        "double": DoubleType(),
+        "float": FloatType(),
+        "int": IntegerType(),
+        "long": LongType(),
+        "string": StringType(),
+    }
+
+    LOGICAL_FIELD_TYPE_MAP: Dict[str, PrimitiveType] = {
+        "date": DateType(),
+        "time-millis": TimeType(),
+        "timestamp-millis": TimestampType(),
+        "uuid": UUIDType(),
+    }
+
+    def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema:
+        """Converts an Apache Avro into an Apache Iceberg schema equivalent
+
+        This expects to have field id's to be encoded in the Avro schema::
+
+            {
+                "type": "record",
+                "name": "manifest_file",
+                "fields": [
+                    {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+                    {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+                ]
+            }
+
+        Example:
+            This converts an Avro schema into a Iceberg schema:
+
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {"name": "manifest_length", "type": "long", "doc": "Total file size in bytes", "field-id": 501}
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=501, name="manifest_length", field_type=LongType(), is_optional=False, doc="Total file size in bytes"
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            avro_schema (Dict[str, Any]): The JSON decoded Avro schema
+
+        Returns:
+            Equivalent Iceberg schema
+        """
+        fields = self._parse_record(avro_schema)
+        return Schema(*fields.fields, schema_id=1)
+
+    def _parse_record(self, avro_field: Dict[str, Any]) -> StructType:
+        fields = [self._parse_field(field) for field in avro_field["fields"]]
+        return StructType(*fields)  # type: ignore
+
+    def _resolve_union(self, type_union: Union[Dict, List, str]) -> Tuple[Union[str, Dict[str, Any]], bool]:
+        """
+        Converts Unions into their type and resolves if the field is optional
+
+        Examples:
+            >>> AvroSchemaConversion()._resolve_union('str')
+            ('str', False)
+            >>> AvroSchemaConversion()._resolve_union(['null', 'str'])
+            ('str', True)
+            >>> AvroSchemaConversion()._resolve_union([{'type': 'str'}])
+            ({'type': 'str'}, False)
+            >>> AvroSchemaConversion()._resolve_union(['null', {'type': 'str'}])
+            ({'type': 'str'}, True)
+
+        Args:
+            type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}
+
+        Returns:
+            A tuple containing the type and nullability
+
+        Raises:
+            TypeError: In the case non-optional union types are encountered
+        """
+        avro_types: Union[Dict, List]
+        if isinstance(type_union, str):
+            # It is a primitive and required
+            return type_union, False
+        elif isinstance(type_union, dict):
+            # It is a context and required
+            return type_union, False
+        else:
+            avro_types = type_union
+
+        is_optional = "null" in avro_types
+
+        # Filter the null value, so we know the actual type
+        avro_types = list(filter(lambda t: t != "null", avro_types))
+
+        if len(avro_types) != 1:
+            raise TypeError("Non-optional types aren't part of the Iceberg specification")
+
+        avro_type = avro_types[0]
+
+        return avro_type, is_optional
+
+    def _resolve_inner_type(
+        self, raw_avro_type: Dict[str, Any], inner_field_name: str, id_field: str
+    ) -> Tuple[IcebergType, bool]:
+        plain_type, element_is_optional = self._resolve_union(raw_avro_type[inner_field_name])
+        inner_field = self._parse_field(plain_type, field_id=raw_avro_type[id_field])
+        if isinstance(inner_field, NestedField):
+            return inner_field.field_type, element_is_optional
+        return inner_field, element_is_optional
+
+    def _parse_field(self, field: Union[str, Dict[str, Any]], field_id: Optional[int] = None) -> IcebergType:
+        """
+        Recursively walks through the Schema, constructing the Iceberg schema
+
+        Examples:
+            >>> avro_schema = AvroSchemaConversion().avro_to_iceberg({
+            ...     "type": "record",
+            ...     "name": "manifest_file",
+            ...     "fields": [
+            ...         {"name": "manifest_path", "type": "string", "doc": "Location URI with FS scheme", "field-id": 500},
+            ...         {
+            ...             "name": "partitions",
+            ...                "type": [
+            ...                "null",
+            ...                {
+            ...                        "type": "array",
+            ...                        "items": {
+            ...                            "type": "record",
+            ...                            "name": "r508",
+            ...                            "fields": [
+            ...                                {
+            ...                                    "name": "contains_null",
+            ...                                    "type": "boolean",
+            ...                                    "doc": "True if any file has a null partition value",
+            ...                                    "field-id": 509,
+            ...                                },
+            ...                                {
+            ...                                    "name": "contains_nan",
+            ...                                    "type": ["null", "boolean"],
+            ...                                    "doc": "True if any file has a NaN partition value",
+            ...                                    "default": None,
+            ...                                    "field-id": 518,
+            ...                                },
+            ...                            ],
+            ...                        },
+            ...                        "element-id": 508,
+            ...                    },
+            ...                ],
+            ...                "doc": "Summary for each partition",
+            ...                "default": None,
+            ...                "field-id": 507,
+            ...            },
+            ...     ]
+            ... })
+            >>> iceberg_schema = Schema(
+            ...     NestedField(
+            ...         field_id=500, name="manifest_path", field_type=StringType(), is_optional=False, doc="Location URI with FS scheme"
+            ...     ),
+            ...     NestedField(
+            ...         field_id=507,
+            ...         name="partitions",
+            ...         field_type=ListType(
+            ...             element_id=508,
+            ...             element_type=StructType(
+            ...                 NestedField(
+            ...                     field_id=509,
+            ...                     name="contains_null",
+            ...                     field_type=BooleanType(),
+            ...                     is_optional=False,
+            ...                     doc="True if any file has a null partition value",
+            ...                 ),
+            ...                 NestedField(
+            ...                     field_id=518,
+            ...                     name="contains_nan",
+            ...                     field_type=BooleanType(),
+            ...                     is_optional=True,
+            ...                     doc="True if any file has a NaN partition value",
+            ...                 )
+            ...             ),
+            ...             element_is_optional=False
+            ...         ),
+            ...         is_optional=True,
+            ...         doc="Summary for each partition",
+            ...     ),
+            ...     schema_id=1
+            ... )
+            >>> avro_schema == iceberg_schema
+            True
+
+        Args:
+            field: The Avro field
+            field_id: Ability to override the field_id when it is provided from up in the tree (in the case of a list or map)
+
+        Returns:
+            The equivalent IcebergType
+        """
+        # In the case of a primitive field
+        if isinstance(field, str):
+            return AvroSchemaConversion.PRIMITIVE_FIELD_TYPE_MAP[field]

Review Comment:
   Yes, that's much cleaner indeed. The method grew a bit over time, but I really like the suggestion of decoupling the field and type. I've updated 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