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

[GitHub] [iceberg] Fokko opened a new pull request, #7782: Python: Bump to Pydantic v2

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

   (no comment)


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

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

For queries about this service, please contact Infrastructure at:
users@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 #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/types.py:
##########
@@ -265,7 +311,7 @@ def __str__(self) -> str:
 
     def __getnewargs__(self) -> Tuple[int, str, IcebergType, bool, Optional[str]]:
         """A magic function for pickling the NestedField class."""
-        return (self.field_id, self.name, self.field_type, self.required, self.doc)
+        return self.field_id, self.name, self.field_type, self.required, self.doc

Review Comment:
   No, but my IDE was bugging me :)



##########
python/pyiceberg/table/metadata.py:
##########
@@ -28,11 +30,11 @@
 
 from pydantic import Field
 from pydantic import ValidationError as PydanticValidationError
-from pydantic import root_validator
+from pydantic import model_validator
 from typing_extensions import Annotated
 
 from pyiceberg.exceptions import ValidationError
-from pyiceberg.partitioning import PartitionSpec, assign_fresh_partition_spec_ids
+from pyiceberg.partitioning import PARTITION_FIELD_ID_START, PartitionSpec, assign_fresh_partition_spec_ids

Review Comment:
   Exactly, the `_` means that it shouldn't be used outside of the class.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/table/metadata.py:
##########
@@ -42,55 +45,79 @@
 from pyiceberg.typedef import EMPTY_DICT, IcebergBaseModel, Properties
 from pyiceberg.utils.datetime import datetime_to_millis
 
-CURRENT_SNAPSHOT_ID = "current_snapshot_id"
-CURRENT_SCHEMA_ID = "current_schema_id"
+CURRENT_SNAPSHOT_ID = "current-snapshot-id"
+CURRENT_SCHEMA_ID = "current-schema-id"
 SCHEMAS = "schemas"
-DEFAULT_SPEC_ID = "default_spec_id"
-PARTITION_SPEC = "partition_spec"
-PARTITION_SPECS = "partition_specs"
-SORT_ORDERS = "sort_orders"
+DEFAULT_SPEC_ID = "default-spec-id"
+PARTITION_SPEC = "partition-spec"
+PARTITION_SPECS = "partition-specs"
+SORT_ORDERS = "sort-orders"
+LAST_PARTITION_ID = "last-partition-id"
+LAST_ASSIGNED_FIELD_ID = "last-assigned-field-id"
 REFS = "refs"
+SPEC_ID = "spec-id"
+FIELD_ID = "field-id"
+FIELDS = "fields"
 
 INITIAL_SEQUENCE_NUMBER = 0
 INITIAL_SPEC_ID = 0
 DEFAULT_SCHEMA_ID = 0
 
 
-def check_schemas(values: Dict[str, Any]) -> Dict[str, Any]:
+def cleanup_snapshot_id(data: Dict[str, Any]) -> Dict[str, Any]:
+    """Runs before validation."""
+    if CURRENT_SNAPSHOT_ID in data and data[CURRENT_SNAPSHOT_ID] == -1:
+        # We treat -1 and None the same, by cleaning this up
+        # in a pre-validator, we can simplify the logic later on
+        data[CURRENT_SNAPSHOT_ID] = None
+    return data
+
+
+def check_schemas(table_metadata: TableMetadata) -> TableMetadata:

Review Comment:
   This is also because the validators are refactored quite a bit.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/types.py:
##########
@@ -34,27 +34,33 @@
 import re
 from typing import (
     Any,
+    Callable,
     ClassVar,
     Dict,
     Generator,
     Literal,
     Optional,
     Tuple,
+    Type,
+    TypeVar,
 )
 
-from pydantic import Field, PrivateAttr
-from pydantic.typing import AnyCallable
+from pydantic import (
+    Field,
+    PrivateAttr,
+    RootModel,
+    model_validator,
+)
 
 from pyiceberg.typedef import IcebergBaseModel
 from pyiceberg.utils.parsing import ParseNumberFromBrackets
-from pyiceberg.utils.singleton import Singleton
 
 DECIMAL_REGEX = re.compile(r"decimal\((\d+),\s*(\d+)\)")
 FIXED = "fixed"
 FIXED_PARSER = ParseNumberFromBrackets(FIXED)
 
 
-class IcebergType(IcebergBaseModel, Singleton):
+class IcebergType(IcebergBaseModel):

Review Comment:
   Is this a change we should apply anyway since not all `IcebergType` subclasses are `Singleton`?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/types.py:
##########
@@ -133,32 +176,34 @@ class FixedType(PrimitiveType):
         False
     """
 
-    __root__: str = Field()
-    _len: int = PrivateAttr()
+    root: int = Field()

Review Comment:
   This is because it serializes to a single value instead of a dict, so we have to map it onto root.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/partitioning.py:
##########
@@ -23,15 +25,21 @@
     Tuple,
 )
 
-from pydantic import Field
+from pydantic import (
+    BeforeValidator,
+    Field,
+    PlainSerializer,
+    WithJsonSchema,
+)
+from typing_extensions import Annotated
 
 from pyiceberg.schema import Schema
-from pyiceberg.transforms import Transform
+from pyiceberg.transforms import Transform, _deserialize_transform
 from pyiceberg.typedef import IcebergBaseModel
 from pyiceberg.types import NestedField, StructType
 
 INITIAL_PARTITION_SPEC_ID = 0
-_PARTITION_DATA_ID_START: int = 1000
+PARTITION_FIELD_ID_START: int = 1000

Review Comment:
   When we read a V1 spec, we use this to set the `last-partition-id` on the metadata:
   ```python
           data[LAST_PARTITION_ID] = max(
               [field.get(FIELD_ID) for spec in data[PARTITION_SPECS] for field in spec[FIELDS]], default=PARTITION_FIELD_ID_START
           )
   ```
   This is outside of the package, so I thought it would make sense to make the field public.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7782: Python: Bump to Pydantic v2

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

   +1 overall. I don't see any blockers once tests are passing. There was one string that looked suspicious since it wasn't translated from underscores to dashes.


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

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

For queries about this service, please contact Infrastructure at:
users@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 #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/types.py:
##########
@@ -167,44 +212,45 @@ class DecimalType(PrimitiveType):
         True
     """
 
-    __root__: str = Field()
+    root: Tuple[int, int]

Review Comment:
   Similar here, it's odd to keep the type state as "root" unless this is something pydantic requires.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7782: Python: Bump to Pydantic v2

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


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

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

For queries about this service, please contact Infrastructure at:
users@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 #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/table/metadata.py:
##########
@@ -42,55 +45,79 @@
 from pyiceberg.typedef import EMPTY_DICT, IcebergBaseModel, Properties
 from pyiceberg.utils.datetime import datetime_to_millis
 
-CURRENT_SNAPSHOT_ID = "current_snapshot_id"
-CURRENT_SCHEMA_ID = "current_schema_id"
+CURRENT_SNAPSHOT_ID = "current-snapshot-id"
+CURRENT_SCHEMA_ID = "current-schema-id"
 SCHEMAS = "schemas"
-DEFAULT_SPEC_ID = "default_spec_id"
-PARTITION_SPEC = "partition_spec"
-PARTITION_SPECS = "partition_specs"
-SORT_ORDERS = "sort_orders"
+DEFAULT_SPEC_ID = "default-spec-id"
+PARTITION_SPEC = "partition-spec"
+PARTITION_SPECS = "partition-specs"
+SORT_ORDERS = "sort-orders"
+LAST_PARTITION_ID = "last-partition-id"
+LAST_ASSIGNED_FIELD_ID = "last-assigned-field-id"
 REFS = "refs"
+SPEC_ID = "spec-id"
+FIELD_ID = "field-id"
+FIELDS = "fields"
 
 INITIAL_SEQUENCE_NUMBER = 0
 INITIAL_SPEC_ID = 0
 DEFAULT_SCHEMA_ID = 0
 
 
-def check_schemas(values: Dict[str, Any]) -> Dict[str, Any]:
+def cleanup_snapshot_id(data: Dict[str, Any]) -> Dict[str, Any]:
+    """Runs before validation."""
+    if CURRENT_SNAPSHOT_ID in data and data[CURRENT_SNAPSHOT_ID] == -1:
+        # We treat -1 and None the same, by cleaning this up
+        # in a pre-validator, we can simplify the logic later on
+        data[CURRENT_SNAPSHOT_ID] = None
+    return data
+
+
+def check_schemas(table_metadata: TableMetadata) -> TableMetadata:

Review Comment:
   The `before` accept an dict with the unprocessed entries, the `after` an actual metadata object. I've switched the ordering because I think it makes sense to first have the `before`'s, and then the `after`'s



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7782: Python: Bump to Pydantic v2

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


##########
python/tests/table/test_metadata.py:
##########
@@ -98,19 +97,6 @@ def test_from_dict_v2_parse_raw(example_table_metadata_v2: Dict[str, Any]) -> No
     TableMetadataUtil.parse_raw(json.dumps(example_table_metadata_v2))
 
 
-@pytest.mark.parametrize(
-    "table_metadata, expected_version",
-    [
-        (EXAMPLE_TABLE_METADATA_V1, 1),
-        (EXAMPLE_TABLE_METADATA_V2, 2),
-    ],
-)
-def test_table_metadata_factory(table_metadata: Dict[str, Any], expected_version: int) -> None:

Review Comment:
   The factory was not needed with V2, so it has been removed. This hasn't been released, so we're not breaking any compatibility.



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

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

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


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


[GitHub] [iceberg] pdames commented on pull request #7782: Python: Bump to Pydantic v2

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

   @Fokko and @rdblue - unfortunately, this has broken our in-progress integration of PyIceberg with Ray on the [Iceberg Branch of DeltaCAT](https://github.com/ray-project/deltacat/tree/iceberg) due to https://github.com/ray-project/ray/issues/37019. Happy to work on both sides to find a path forward, but seems like we'll need to pin to a prior commit of PyIceberg that depends on Pydantic for continued Ray 2.X interoperability in the mean 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 #7782: Python: Bump to Pydantic v2

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


##########
python/tests/table/test_metadata.py:
##########
@@ -98,19 +97,6 @@ def test_from_dict_v2_parse_raw(example_table_metadata_v2: Dict[str, Any]) -> No
     TableMetadataUtil.parse_raw(json.dumps(example_table_metadata_v2))
 
 
-@pytest.mark.parametrize(
-    "table_metadata, expected_version",
-    [
-        (EXAMPLE_TABLE_METADATA_V1, 1),
-        (EXAMPLE_TABLE_METADATA_V2, 2),
-    ],
-)
-def test_table_metadata_factory(table_metadata: Dict[str, Any], expected_version: int) -> None:

Review Comment:
   The factory was not 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 pull request #7782: Python: Bump to Pydantic v2

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

   Did some local tests, and it looks great:
   <img width="1624" alt="image" src="https://github.com/apache/iceberg/assets/1134248/9c88b33a-7dfa-4c40-8a91-f0c3fb6af8a7">
   


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

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

For queries about this service, please contact Infrastructure at:
users@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 #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/partitioning.py:
##########
@@ -23,15 +25,21 @@
     Tuple,
 )
 
-from pydantic import Field
+from pydantic import (
+    BeforeValidator,
+    Field,
+    PlainSerializer,
+    WithJsonSchema,
+)
+from typing_extensions import Annotated
 
 from pyiceberg.schema import Schema
-from pyiceberg.transforms import Transform
+from pyiceberg.transforms import Transform, _deserialize_transform
 from pyiceberg.typedef import IcebergBaseModel
 from pyiceberg.types import NestedField, StructType
 
 INITIAL_PARTITION_SPEC_ID = 0
-_PARTITION_DATA_ID_START: int = 1000
+PARTITION_FIELD_ID_START: int = 1000

Review Comment:
   Why is this needed?



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -781,7 +781,7 @@ def _task_to_table(
             schema_raw = metadata.get(ICEBERG_SCHEMA)
         # TODO: if field_ids are not present, Name Mapping should be implemented to look them up in the table schema,
         #  see https://github.com/apache/iceberg/issues/7451
-        file_schema = Schema.parse_raw(schema_raw) if schema_raw is not None else pyarrow_to_schema(physical_schema)
+        file_schema = Schema.model_validate_json(schema_raw) if schema_raw is not None else pyarrow_to_schema(physical_schema)

Review Comment:
   Nit: It uses "validate" instead of "parse"? That seems weird to me.



##########
python/pyiceberg/types.py:
##########
@@ -390,25 +446,49 @@ def __init__(
         value_required: bool = True,
         **data: Any,
     ):
-        data["key_id"] = key_id or data["key-id"]
-        data["key_type"] = key_type or data["key"]
-        data["value_id"] = value_id or data["value-id"]
-        data["value_type"] = value_type or data["value"]
-        data["value_required"] = value_required if value_required is not None else data["value_required"]
-
-        data["key_field"] = NestedField(name="key", field_id=data["key_id"], field_type=data["key_type"], required=True)
-        data["value_field"] = NestedField(
-            name="value", field_id=data["value_id"], field_type=data["value_type"], required=data["value_required"]
-        )
+        data["key-id"] = data["key-id"] if "key-id" in data else key_id
+        data["key"] = data["key"] if "key" in data else key_type
+        data["value-id"] = data["value-id"] if "value-id" in data else value_id
+        data["value"] = data["value"] if "value" in data else value_type
+        data["value_required"] = data["value_required"] if "value_required" in data else value_required

Review Comment:
   Should this be `value-required`?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -781,7 +781,7 @@ def _task_to_table(
             schema_raw = metadata.get(ICEBERG_SCHEMA)
         # TODO: if field_ids are not present, Name Mapping should be implemented to look them up in the table schema,
         #  see https://github.com/apache/iceberg/issues/7451
-        file_schema = Schema.parse_raw(schema_raw) if schema_raw is not None else pyarrow_to_schema(physical_schema)
+        file_schema = Schema.model_validate_json(schema_raw) if schema_raw is not None else pyarrow_to_schema(physical_schema)

Review Comment:
   From the migration guide: https://docs.pydantic.dev/latest/migration/
   <img width="682" alt="image" src="https://github.com/apache/iceberg/assets/1134248/0c8b1383-9104-4375-b8e8-9bdd04185338">
   



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/types.py:
##########
@@ -133,32 +176,34 @@ class FixedType(PrimitiveType):
         False
     """
 
-    __root__: str = Field()
-    _len: int = PrivateAttr()
+    root: int = Field()

Review Comment:
   Why is the length called `root` now? Can we call it `len` or similar instead? I think it is confusing this way.



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/types.py:
##########
@@ -232,31 +238,40 @@ class NestedField(IcebergType):
 
     field_id: int = Field(alias="id")
     name: str = Field()
-    field_type: IcebergType = Field(alias="type")
+    field_type: IcebergSubTypes = Field(alias="type")

Review Comment:
   This has been fixed in https://github.com/pydantic/pydantic/issues/6403



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/transforms.py:
##########
@@ -106,47 +100,37 @@ def _transform_literal(func: Callable[[L], L], lit: Literal[L]) -> Literal[L]:
     return literal(func(lit.value))
 
 
-class Transform(IcebergBaseModel, ABC, Generic[S, T]):
+def _deserialize_transform(v: Any) -> Any:
+    if isinstance(v, str):
+        if v == IDENTITY:
+            return IdentityTransform()
+        elif v == VOID:
+            return VoidTransform()
+        elif v.startswith(BUCKET):
+            return BucketTransform(num_buckets=BUCKET_PARSER.match(v))
+        elif v.startswith(TRUNCATE):
+            return TruncateTransform(width=TRUNCATE_PARSER.match(v))
+        elif v == YEAR:
+            return YearTransform()
+        elif v == MONTH:
+            return MonthTransform()
+        elif v == DAY:
+            return DayTransform()
+        elif v == HOUR:
+            return HourTransform()
+        else:
+            return UnknownTransform(transform=v)
+    return v
+
+
+class Transform(IcebergRootModel[str], ABC, Generic[S, T]):
     """Transform base class for concrete transforms.
 
     A base class to transform values and project predicates on partition values.
     This class is not used directly. Instead, use one of module method to create the child classes.
     """
 
-    __root__: str = Field()
-
-    @classmethod
-    def __get_validators__(cls) -> Generator[AnyCallable, None, None]:
-        """Called to validate the input of the Transform class."""
-        # one or more validators may be yielded which will be called in the
-        # order to validate the input, each validator will receive as an input
-        # the value returned from the previous validator
-        yield cls.validate
-
-    @classmethod
-    def validate(cls, v: Any) -> IcebergBaseModel:

Review Comment:
   This changed in 2.0: https://docs.pydantic.dev/latest/migration/#validate_arguments-has-been-renamed-to-validate_call



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7782: Python: Bump to Pydantic v2

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

   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] Fokko commented on a diff in pull request #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -365,8 +365,8 @@ class AssertDefaultSortOrderId(TableRequirement):
 
 class CommitTableRequest(IcebergBaseModel):
     identifier: Identifier = Field()
-    requirements: List[TableRequirement] = Field(default_factory=list)
-    updates: List[TableUpdate] = Field(default_factory=list)
+    requirements: List[SerializeAsAny[TableRequirement]] = Field(default_factory=list)
+    updates: List[SerializeAsAny[TableUpdate]] = Field(default_factory=list)

Review Comment:
   I also had to learn this by raising an issue: https://github.com/pydantic/pydantic/issues/6403#issuecomment-1620252270
   
   It turns out that you need to wrap this when you're serializing an object, that might have subclasses. This way it gets correctly serialized.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7782: Python: Bump to Pydantic v2

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

   I also did some benchmarking, and it is faster:
   
   This branch:
   ```
   ➜  python git:(fd-bump-pydantic) ✗ hyperfine --warmup 1 "python3 tests/test_benchmark.py" 
   Benchmark 1: python3 tests/test_benchmark.py
     Time (mean ± σ):      1.462 s ±  0.006 s    [User: 1.433 s, System: 0.026 s]
     Range (min … max):    1.454 s …  1.472 s    10 runs
   ```
   
   Current master:
   ```
   ➜  python git:(master) ✗ hyperfine --warmup 1 "python3 tests/test_benchmark.py"
   Benchmark 1: python3 tests/test_benchmark.py
     Time (mean ± σ):      1.961 s ±  0.011 s    [User: 1.933 s, System: 0.024 s]
     Range (min … max):    1.944 s …  1.978 s    10 runs
   ```
   
   This parses 10k json strings:
   ```python
   import json
   
   from pyiceberg.table.metadata import TableMetadataUtil
   
   EXAMPLE_TABLE_METADATA_V2 = {
       "format-version": 2,
       "table-uuid": "9c12d441-03fe-4693-9a96-a0705ddf69c1",
       "location": "s3://bucket/test/location",
       "last-sequence-number": 34,
       "last-updated-ms": 1602638573590,
       "last-column-id": 3,
       "current-schema-id": 1,
       "schemas": [
           {"type": "struct", "schema-id": 0, "fields": [{"id": 1, "name": "x", "required": True, "type": "long"}]},
           {
               "type": "struct",
               "schema-id": 1,
               "identifier-field-ids": [1, 2],
               "fields": [
                   {"id": 1, "name": "x", "required": True, "type": "long"},
                   {"id": 2, "name": "y", "required": True, "type": "long", "doc": "comment"},
                   {"id": 3, "name": "z", "required": True, "type": "long"},
               ],
           },
       ],
       "default-spec-id": 0,
       "partition-specs": [{"spec-id": 0, "fields": [{"name": "x", "transform": "identity", "source-id": 1, "field-id": 1000}]}],
       "last-partition-id": 1000,
       "default-sort-order-id": 3,
       "sort-orders": [
           {
               "order-id": 3,
               "fields": [
                   {"transform": "identity", "source-id": 2, "direction": "asc", "null-order": "nulls-first"},
                   {"transform": "bucket[4]", "source-id": 3, "direction": "desc", "null-order": "nulls-last"},
               ],
           }
       ],
       "properties": {"read.split.target.size": "134217728"},
       "current-snapshot-id": 3055729675574597004,
       "snapshots": [
           {
               "snapshot-id": 3051729675574597004,
               "timestamp-ms": 1515100955770,
               "sequence-number": 0,
               "summary": {"operation": "append"},
               "manifest-list": "s3://a/b/1.avro",
           },
           {
               "snapshot-id": 3055729675574597004,
               "parent-snapshot-id": 3051729675574597004,
               "timestamp-ms": 1555100955770,
               "sequence-number": 1,
               "summary": {"operation": "append"},
               "manifest-list": "s3://a/b/2.avro",
               "schema-id": 1,
           },
       ],
       "snapshot-log": [
           {"snapshot-id": 3051729675574597004, "timestamp-ms": 1515100955770},
           {"snapshot-id": 3055729675574597004, "timestamp-ms": 1555100955770},
       ],
       "metadata-log": [{"metadata-file": "s3://bucket/.../v1.json", "timestamp-ms": 1515100}],
       "refs": {"test": {"snapshot-id": 3051729675574597004, "type": "tag", "max-ref-age-ms": 10000000}},
   }
   
   
   metadata_v2 = json.dumps(EXAMPLE_TABLE_METADATA_V2)
   
   for _ in range(10000):
       TableMetadataUtil.parse_raw(metadata_v2)
   ```


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

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

For queries about this service, please contact Infrastructure at:
users@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 #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/transforms.py:
##########
@@ -105,17 +104,17 @@ def _transform_literal(func: Callable[[L], L], lit: Literal[L]) -> Literal[L]:
     return literal(func(lit.value))
 
 
-class Transform(IcebergBaseModel, ABC, Generic[S, T]):
+class Transform(RootModel, ABC):

Review Comment:
   Why does this lose the generics?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/table/metadata.py:
##########
@@ -42,55 +45,79 @@
 from pyiceberg.typedef import EMPTY_DICT, IcebergBaseModel, Properties
 from pyiceberg.utils.datetime import datetime_to_millis
 
-CURRENT_SNAPSHOT_ID = "current_snapshot_id"
-CURRENT_SCHEMA_ID = "current_schema_id"
+CURRENT_SNAPSHOT_ID = "current-snapshot-id"
+CURRENT_SCHEMA_ID = "current-schema-id"
 SCHEMAS = "schemas"
-DEFAULT_SPEC_ID = "default_spec_id"
-PARTITION_SPEC = "partition_spec"
-PARTITION_SPECS = "partition_specs"
-SORT_ORDERS = "sort_orders"
+DEFAULT_SPEC_ID = "default-spec-id"
+PARTITION_SPEC = "partition-spec"
+PARTITION_SPECS = "partition-specs"
+SORT_ORDERS = "sort-orders"
+LAST_PARTITION_ID = "last-partition-id"
+LAST_ASSIGNED_FIELD_ID = "last-assigned-field-id"
 REFS = "refs"
+SPEC_ID = "spec-id"
+FIELD_ID = "field-id"
+FIELDS = "fields"
 
 INITIAL_SEQUENCE_NUMBER = 0
 INITIAL_SPEC_ID = 0
 DEFAULT_SCHEMA_ID = 0
 
 
-def check_schemas(values: Dict[str, Any]) -> Dict[str, Any]:
+def cleanup_snapshot_id(data: Dict[str, Any]) -> Dict[str, Any]:
+    """Runs before validation."""
+    if CURRENT_SNAPSHOT_ID in data and data[CURRENT_SNAPSHOT_ID] == -1:
+        # We treat -1 and None the same, by cleaning this up
+        # in a pre-validator, we can simplify the logic later on
+        data[CURRENT_SNAPSHOT_ID] = None
+    return data
+
+
+def check_schemas(table_metadata: TableMetadata) -> TableMetadata:

Review Comment:
   Changed a lot of these validators to `after`, instead of `before`. The `before` would validate on the Python dictionaries without any alias'es applied. The `after` uses the metadata object itself, which is more safe and less error prone.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/table/snapshots.py:
##########
@@ -59,34 +58,19 @@ class Summary(IcebergBaseModel):
     like snapshot expiration, to skip processing certain snapshots.
     """
 
-    __root__: Dict[str, Union[str, Operation]]
+    operation: Operation = Field()
     _additional_properties: Dict[str, str] = PrivateAttr()
 
-    @root_validator
-    def check_operation(cls, values: Dict[str, Dict[str, Union[str, Operation]]]) -> Dict[str, Dict[str, Union[str, Operation]]]:
-        if operation := values["__root__"].get(OPERATION):
-            if isinstance(operation, str):
-                values["__root__"][OPERATION] = Operation(operation.lower())
-        else:
-            raise ValueError("Operation not set")
-        return values
-
-    def __init__(
-        self, operation: Optional[Operation] = None, __root__: Optional[Dict[str, Union[str, Operation]]] = None, **data: Any
-    ) -> None:
-        super().__init__(__root__={"operation": operation, **data} if not __root__ else __root__)
-        self._additional_properties = {
-            k: v for k, v in self.__root__.items() if k != OPERATION  # type: ignore # We know that they are all string, and we don't want to check
-        }
+    def __init__(self, operation: Optional[Operation] = None, **data: Any) -> None:

Review Comment:
   While `operation` was not originally written, it should always be there so we don't necessarily need it to be `Optional`.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/table/metadata.py:
##########
@@ -42,55 +45,79 @@
 from pyiceberg.typedef import EMPTY_DICT, IcebergBaseModel, Properties
 from pyiceberg.utils.datetime import datetime_to_millis
 
-CURRENT_SNAPSHOT_ID = "current_snapshot_id"
-CURRENT_SCHEMA_ID = "current_schema_id"
+CURRENT_SNAPSHOT_ID = "current-snapshot-id"
+CURRENT_SCHEMA_ID = "current-schema-id"
 SCHEMAS = "schemas"
-DEFAULT_SPEC_ID = "default_spec_id"
-PARTITION_SPEC = "partition_spec"
-PARTITION_SPECS = "partition_specs"
-SORT_ORDERS = "sort_orders"
+DEFAULT_SPEC_ID = "default-spec-id"
+PARTITION_SPEC = "partition-spec"
+PARTITION_SPECS = "partition-specs"
+SORT_ORDERS = "sort-orders"
+LAST_PARTITION_ID = "last-partition-id"
+LAST_ASSIGNED_FIELD_ID = "last-assigned-field-id"
 REFS = "refs"
+SPEC_ID = "spec-id"
+FIELD_ID = "field-id"
+FIELDS = "fields"
 
 INITIAL_SEQUENCE_NUMBER = 0
 INITIAL_SPEC_ID = 0
 DEFAULT_SCHEMA_ID = 0
 
 
-def check_schemas(values: Dict[str, Any]) -> Dict[str, Any]:
+def cleanup_snapshot_id(data: Dict[str, Any]) -> Dict[str, Any]:
+    """Runs before validation."""
+    if CURRENT_SNAPSHOT_ID in data and data[CURRENT_SNAPSHOT_ID] == -1:
+        # We treat -1 and None the same, by cleaning this up
+        # in a pre-validator, we can simplify the logic later on
+        data[CURRENT_SNAPSHOT_ID] = None
+    return data
+
+
+def check_schemas(table_metadata: TableMetadata) -> TableMetadata:

Review Comment:
   I like how we use the same keys now.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/transforms.py:
##########
@@ -106,47 +100,37 @@ def _transform_literal(func: Callable[[L], L], lit: Literal[L]) -> Literal[L]:
     return literal(func(lit.value))
 
 
-class Transform(IcebergBaseModel, ABC, Generic[S, T]):
+def _deserialize_transform(v: Any) -> Any:

Review Comment:
   I've renamed this to `parse_transform`



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/types.py:
##########
@@ -167,44 +212,45 @@ class DecimalType(PrimitiveType):
         True
     """
 
-    __root__: str = Field()
+    root: Tuple[int, int]

Review Comment:
   This is required. Because this doesn't (de)serializes to dict, it has to map onto the root.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -781,7 +781,7 @@ def _task_to_table(
             schema_raw = metadata.get(ICEBERG_SCHEMA)
         # TODO: if field_ids are not present, Name Mapping should be implemented to look them up in the table schema,
         #  see https://github.com/apache/iceberg/issues/7451
-        file_schema = Schema.parse_raw(schema_raw) if schema_raw is not None else pyarrow_to_schema(physical_schema)
+        file_schema = Schema.model_validate_json(schema_raw) if schema_raw is not None else pyarrow_to_schema(physical_schema)

Review Comment:
   I agree, I would also prefer `parse`



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/partitioning.py:
##########
@@ -23,15 +25,21 @@
     Tuple,
 )
 
-from pydantic import Field
+from pydantic import (
+    BeforeValidator,
+    Field,
+    PlainSerializer,
+    WithJsonSchema,
+)
+from typing_extensions import Annotated
 
 from pyiceberg.schema import Schema
-from pyiceberg.transforms import Transform
+from pyiceberg.transforms import Transform, _deserialize_transform
 from pyiceberg.typedef import IcebergBaseModel
 from pyiceberg.types import NestedField, StructType
 
 INITIAL_PARTITION_SPEC_ID = 0
-_PARTITION_DATA_ID_START: int = 1000
+PARTITION_FIELD_ID_START: int = 1000

Review Comment:
   When we read a V1 spec, we use this to set the `last-partition-id` on the metadata:
   ```
           data[LAST_PARTITION_ID] = max(
               [field.get(FIELD_ID) for spec in data[PARTITION_SPECS] for field in spec[FIELDS]], default=PARTITION_FIELD_ID_START
           )
   ```
   This is outside of the package, so I thought it would make sense to make public.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/table/metadata.py:
##########
@@ -42,55 +45,79 @@
 from pyiceberg.typedef import EMPTY_DICT, IcebergBaseModel, Properties
 from pyiceberg.utils.datetime import datetime_to_millis
 
-CURRENT_SNAPSHOT_ID = "current_snapshot_id"
-CURRENT_SCHEMA_ID = "current_schema_id"
+CURRENT_SNAPSHOT_ID = "current-snapshot-id"
+CURRENT_SCHEMA_ID = "current-schema-id"
 SCHEMAS = "schemas"
-DEFAULT_SPEC_ID = "default_spec_id"
-PARTITION_SPEC = "partition_spec"
-PARTITION_SPECS = "partition_specs"
-SORT_ORDERS = "sort_orders"
+DEFAULT_SPEC_ID = "default-spec-id"
+PARTITION_SPEC = "partition-spec"
+PARTITION_SPECS = "partition-specs"
+SORT_ORDERS = "sort-orders"
+LAST_PARTITION_ID = "last-partition-id"
+LAST_ASSIGNED_FIELD_ID = "last-assigned-field-id"
 REFS = "refs"
+SPEC_ID = "spec-id"
+FIELD_ID = "field-id"
+FIELDS = "fields"
 
 INITIAL_SEQUENCE_NUMBER = 0
 INITIAL_SPEC_ID = 0
 DEFAULT_SCHEMA_ID = 0
 
 
-def check_schemas(values: Dict[str, Any]) -> Dict[str, Any]:
+def cleanup_snapshot_id(data: Dict[str, Any]) -> Dict[str, Any]:
+    """Runs before validation."""
+    if CURRENT_SNAPSHOT_ID in data and data[CURRENT_SNAPSHOT_ID] == -1:
+        # We treat -1 and None the same, by cleaning this up
+        # in a pre-validator, we can simplify the logic later on
+        data[CURRENT_SNAPSHOT_ID] = None
+    return data
+
+
+def check_schemas(table_metadata: TableMetadata) -> TableMetadata:

Review Comment:
   Yes, and it also handles the aliases. For example, for the `NestedField` we both accept `field_id` and `id` as the `id`. But since `id` is a reserved keyword we map `id` to `field_id`. In the `after` validator, we make sure that `id` is converted to `field_id`, which makes it less error-prone :)



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/io/pyarrow.py:
##########
@@ -781,7 +781,7 @@ def _task_to_table(
             schema_raw = metadata.get(ICEBERG_SCHEMA)
         # TODO: if field_ids are not present, Name Mapping should be implemented to look them up in the table schema,
         #  see https://github.com/apache/iceberg/issues/7451
-        file_schema = Schema.parse_raw(schema_raw) if schema_raw is not None else pyarrow_to_schema(physical_schema)
+        file_schema = Schema.model_validate_json(schema_raw) if schema_raw is not None else pyarrow_to_schema(physical_schema)

Review Comment:
   Yeah, it just doesn't seem to describe what the method is doing. It's not a validation. It's producing a different object.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/utils/parsing.py:
##########
@@ -32,4 +34,4 @@ def match(self, str_repr: str) -> int:
         matches = self.regex.search(str_repr)
         if matches:
             return int(matches.group(1))
-        raise ValueError(f"Could not match {str_repr}, expected format {self.prefix}[22]")
+        raise ValidationError(f"Could not match {str_repr}, expected format {self.prefix}[22]")

Review Comment:
   Should `22` be hard-coded 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 #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/types.py:
##########
@@ -34,27 +34,33 @@
 import re
 from typing import (
     Any,
+    Callable,
     ClassVar,
     Dict,
     Generator,
     Literal,
     Optional,
     Tuple,
+    Type,
+    TypeVar,
 )
 
-from pydantic import Field, PrivateAttr
-from pydantic.typing import AnyCallable
+from pydantic import (
+    Field,
+    PrivateAttr,
+    RootModel,
+    model_validator,
+)
 
 from pyiceberg.typedef import IcebergBaseModel
 from pyiceberg.utils.parsing import ParseNumberFromBrackets
-from pyiceberg.utils.singleton import Singleton
 
 DECIMAL_REGEX = re.compile(r"decimal\((\d+),\s*(\d+)\)")
 FIXED = "fixed"
 FIXED_PARSER = ParseNumberFromBrackets(FIXED)
 
 
-class IcebergType(IcebergBaseModel, Singleton):
+class IcebergType(IcebergBaseModel):

Review Comment:
   (I've removed this singleton temporarily to make sure that it didn't interfere with Pydantic 2.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 #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/table/metadata.py:
##########
@@ -28,11 +30,11 @@
 
 from pydantic import Field
 from pydantic import ValidationError as PydanticValidationError
-from pydantic import root_validator
+from pydantic import model_validator
 from typing_extensions import Annotated
 
 from pyiceberg.exceptions import ValidationError
-from pyiceberg.partitioning import PartitionSpec, assign_fresh_partition_spec_ids
+from pyiceberg.partitioning import PARTITION_FIELD_ID_START, PartitionSpec, assign_fresh_partition_spec_ids

Review Comment:
   Ah, so this was needed outside of `partitioning` and that's why it is no longer prefixed with an underscore?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/types.py:
##########
@@ -34,27 +34,33 @@
 import re
 from typing import (
     Any,
+    Callable,
     ClassVar,
     Dict,
     Generator,
     Literal,
     Optional,
     Tuple,
+    Type,
+    TypeVar,
 )
 
-from pydantic import Field, PrivateAttr
-from pydantic.typing import AnyCallable
+from pydantic import (
+    Field,
+    PrivateAttr,
+    RootModel,
+    model_validator,
+)
 
 from pyiceberg.typedef import IcebergBaseModel
 from pyiceberg.utils.parsing import ParseNumberFromBrackets
-from pyiceberg.utils.singleton import Singleton
 
 DECIMAL_REGEX = re.compile(r"decimal\((\d+),\s*(\d+)\)")
 FIXED = "fixed"
 FIXED_PARSER = ParseNumberFromBrackets(FIXED)
 
 
-class IcebergType(IcebergBaseModel, Singleton):
+class IcebergType(IcebergBaseModel):

Review Comment:
   Currently, this is the case. The singleton will hash the arguments and if there is already an instance with the same arguments, it will be returned. I'm happy to change this to exclude:
   
   - NestedField
   - StructType
   - MapType
   - ListType
   
   Also, the FixedType, DecimalType, etc could be excluded.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/utils/parsing.py:
##########
@@ -32,4 +34,4 @@ def match(self, str_repr: str) -> int:
         matches = self.regex.search(str_repr)
         if matches:
             return int(matches.group(1))
-        raise ValueError(f"Could not match {str_repr}, expected format {self.prefix}[22]")
+        raise ValidationError(f"Could not match {str_repr}, expected format {self.prefix}[22]")

Review Comment:
   It is an example, it can be arbitrary. The `ParseNumberFromBrackets('ryan')` would accept anything like `ryan[22]`



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/types.py:
##########
@@ -265,7 +311,7 @@ def __str__(self) -> str:
 
     def __getnewargs__(self) -> Tuple[int, str, IcebergType, bool, Optional[str]]:
         """A magic function for pickling the NestedField class."""
-        return (self.field_id, self.name, self.field_type, self.required, self.doc)
+        return self.field_id, self.name, self.field_type, self.required, self.doc

Review Comment:
   Reverted, also for the `{List,Map}Type`



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/table/snapshots.py:
##########
@@ -59,34 +58,19 @@ class Summary(IcebergBaseModel):
     like snapshot expiration, to skip processing certain snapshots.
     """
 
-    __root__: Dict[str, Union[str, Operation]]
+    operation: Operation = Field()
     _additional_properties: Dict[str, str] = PrivateAttr()
 
-    @root_validator
-    def check_operation(cls, values: Dict[str, Dict[str, Union[str, Operation]]]) -> Dict[str, Dict[str, Union[str, Operation]]]:
-        if operation := values["__root__"].get(OPERATION):
-            if isinstance(operation, str):
-                values["__root__"][OPERATION] = Operation(operation.lower())
-        else:
-            raise ValueError("Operation not set")
-        return values
-
-    def __init__(
-        self, operation: Optional[Operation] = None, __root__: Optional[Dict[str, Union[str, Operation]]] = None, **data: Any
-    ) -> None:
-        super().__init__(__root__={"operation": operation, **data} if not __root__ else __root__)
-        self._additional_properties = {
-            k: v for k, v in self.__root__.items() if k != OPERATION  # type: ignore # We know that they are all string, and we don't want to check
-        }
+    def __init__(self, operation: Optional[Operation] = None, **data: Any) -> None:

Review Comment:
   Nice catch, thanks! That's indeed not needed.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/types.py:
##########
@@ -265,7 +311,7 @@ def __str__(self) -> str:
 
     def __getnewargs__(self) -> Tuple[int, str, IcebergType, bool, Optional[str]]:
         """A magic function for pickling the NestedField class."""
-        return (self.field_id, self.name, self.field_type, self.required, self.doc)
+        return self.field_id, self.name, self.field_type, self.required, self.doc

Review Comment:
   Is this change required?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/table/metadata.py:
##########
@@ -217,7 +232,15 @@ class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
     # because bumping the version should be an explicit operation that is up
     # to the owner of the table.
 
-    @root_validator
+    @model_validator(mode="before")
+    def cleanup_snapshot_id(cls, data: Dict[str, Any]) -> Dict[str, Any]:
+        return cleanup_snapshot_id(data)

Review Comment:
   Is this reused? Why have a separate method instead of inlining the logic 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 #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/types.py:
##########
@@ -332,33 +382,46 @@ class ListType(IcebergType):
     class Config:
         fields = {"element_field": {"exclude": True}}
 
-    type: Literal["list"] = "list"
+    type: Literal["list"] = Field(default="list")
     element_id: int = Field(alias="element-id")
-    element_type: IcebergType = Field(alias="element")
+    element_type: SerializeAsAny[IcebergType] = Field(alias="element")
     element_required: bool = Field(alias="element-required", default=True)
-    element_field: NestedField = Field(init=False, repr=False)
+    _element_field: NestedField = PrivateAttr()
+    _hash: int = PrivateAttr()
 
     def __init__(
         self, element_id: Optional[int] = None, element: Optional[IcebergType] = None, element_required: bool = True, **data: Any
     ):
-        data["element_id"] = data["element-id"] if "element-id" in data else element_id
-        data["element_type"] = element or data["element_type"]
-        data["element_required"] = data["element-required"] if "element-required" in data else element_required
-        data["element_field"] = NestedField(
+        data["element-id"] = data["element-id"] if "element-id" in data else element_id
+        data["element"] = element or data["element_type"]
+        data["element-required"] = data["element-required"] if "element-required" in data else element_required
+        super().__init__(**data)
+        self._hash = hash(data.values())
+
+    @cached_property
+    def element_field(self) -> NestedField:

Review Comment:
   I think this is much nicer than pre-creating a private attribute.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/table/metadata.py:
##########
@@ -217,7 +232,15 @@ class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
     # because bumping the version should be an explicit operation that is up
     # to the owner of the table.
 
-    @root_validator
+    @model_validator(mode="before")
+    def cleanup_snapshot_id(cls, data: Dict[str, Any]) -> Dict[str, Any]:
+        return cleanup_snapshot_id(data)

Review Comment:
   Is this reused? Why have a separate method instead of inlining the logic here?



##########
python/pyiceberg/transforms.py:
##########
@@ -106,47 +100,37 @@ def _transform_literal(func: Callable[[L], L], lit: Literal[L]) -> Literal[L]:
     return literal(func(lit.value))
 
 
-class Transform(IcebergBaseModel, ABC, Generic[S, T]):
+def _deserialize_transform(v: Any) -> Any:

Review Comment:
   Minor: This could be public. It seems useful.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/table/metadata.py:
##########
@@ -359,15 +387,6 @@ def check_sort_orders(cls, values: Dict[str, Any]) -> Dict[str, Any]:
 TableMetadata = Annotated[Union[TableMetadataV1, TableMetadataV2], Field(discriminator="format_version")]
 
 
-class TableMetadataFactory(IcebergBaseModel):

Review Comment:
   I've replaced this with a root-model so we don't have to do the `f'{{"table_metadata": {data}}}'`. cc @aless10



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/types.py:
##########
@@ -390,25 +446,49 @@ def __init__(
         value_required: bool = True,
         **data: Any,
     ):
-        data["key_id"] = key_id or data["key-id"]
-        data["key_type"] = key_type or data["key"]
-        data["value_id"] = value_id or data["value-id"]
-        data["value_type"] = value_type or data["value"]
-        data["value_required"] = value_required if value_required is not None else data["value_required"]
-
-        data["key_field"] = NestedField(name="key", field_id=data["key_id"], field_type=data["key_type"], required=True)
-        data["value_field"] = NestedField(
-            name="value", field_id=data["value_id"], field_type=data["value_type"], required=data["value_required"]
-        )
+        data["key-id"] = data["key-id"] if "key-id" in data else key_id
+        data["key"] = data["key"] if "key" in data else key_type
+        data["value-id"] = data["value-id"] if "value-id" in data else value_id
+        data["value"] = data["value"] if "value" in data else value_type
+        data["value_required"] = data["value_required"] if "value_required" in data else value_required

Review Comment:
   Yes, thank you :)



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/table/snapshots.py:
##########
@@ -59,34 +58,19 @@ class Summary(IcebergBaseModel):
     like snapshot expiration, to skip processing certain snapshots.
     """
 
-    __root__: Dict[str, Union[str, Operation]]
+    operation: Operation = Field()
     _additional_properties: Dict[str, str] = PrivateAttr()
 
-    @root_validator

Review Comment:
   A lot of changes here because the additional fields in the operation `dict` are considered properties. Since the `@root_validator` is not there anymore, we completely had to rewrite this. For the better I'd say.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -365,8 +365,8 @@ class AssertDefaultSortOrderId(TableRequirement):
 
 class CommitTableRequest(IcebergBaseModel):
     identifier: Identifier = Field()
-    requirements: List[TableRequirement] = Field(default_factory=list)
-    updates: List[TableUpdate] = Field(default_factory=list)
+    requirements: List[SerializeAsAny[TableRequirement]] = Field(default_factory=list)
+    updates: List[SerializeAsAny[TableUpdate]] = Field(default_factory=list)

Review Comment:
   Can you help me understand what's happening here?



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #7782: Python: Bump to Pydantic v2

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


##########
python/tests/conftest.py:
##########
@@ -311,7 +311,7 @@ def all_avro_types() -> Dict[str, Any]:
             ],
         }
     ],
-    "properties": {"read.split.target.size": 134217728},
+    "properties": {"read.split.target.size": "134217728"},

Review Comment:
   Good catch.



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

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

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


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


[GitHub] [iceberg] blublinsky commented on pull request #7782: Python: Bump to Pydantic v2

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

   We have to be very careful here. Pydantic is also used by KFP and Ray. We are currently on Version: 1.10.4. I am afraid, this update will break the compatibility with other components that we are using


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

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

For queries about this service, please contact Infrastructure at:
users@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 #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/table/snapshots.py:
##########
@@ -128,5 +119,5 @@ class MetadataLogEntry(IcebergBaseModel):
 
 
 class SnapshotLogEntry(IcebergBaseModel):
-    snapshot_id: str = Field(alias="snapshot-id")
+    snapshot_id: int = Field(alias="snapshot-id")

Review Comment:
   Looks like this was incorrect? Should we fix this change in main?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/types.py:
##########
@@ -232,31 +238,40 @@ class NestedField(IcebergType):
 
     field_id: int = Field(alias="id")
     name: str = Field()
-    field_type: IcebergType = Field(alias="type")
+    field_type: IcebergSubTypes = Field(alias="type")
     required: bool = Field(default=True)
     doc: Optional[str] = Field(default=None, repr=False)
-    initial_default: Any = Field(alias="initial-default", repr=False)
+    initial_default: Optional[Any] = Field(alias="initial-default", default=None, repr=False)
 
     def __init__(
         self,
         field_id: Optional[int] = None,
         name: Optional[str] = None,
-        field_type: Optional[IcebergType] = None,
+        field_type: Optional[Type[IcebergType]] = None,

Review Comment:
   What is happening here? Isn't the type an instance of `IcebergType`?



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #7782: Python: Bump to Pydantic v2

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


##########
python/pyiceberg/types.py:
##########
@@ -232,31 +238,40 @@ class NestedField(IcebergType):
 
     field_id: int = Field(alias="id")
     name: str = Field()
-    field_type: IcebergType = Field(alias="type")
+    field_type: IcebergSubTypes = Field(alias="type")
     required: bool = Field(default=True)
     doc: Optional[str] = Field(default=None, repr=False)
-    initial_default: Any = Field(alias="initial-default", repr=False)
+    initial_default: Optional[Any] = Field(alias="initial-default", default=None, repr=False)
 
     def __init__(
         self,
         field_id: Optional[int] = None,
         name: Optional[str] = None,
-        field_type: Optional[IcebergType] = None,
+        field_type: Optional[Type[IcebergType]] = None,

Review Comment:
   This is incorrect indeed,



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

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

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


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